From 3d595ed5f299ac4f3da671eca8c9dc652672535f Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Wed, 25 Mar 2026 13:38:00 -0400 Subject: [PATCH 01/27] Slow but correct impls --- .../examples/columnar/columnar_support.rs | 670 +++++++----------- 1 file changed, 274 insertions(+), 396 deletions(-) diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index ced700bec..b6a67af25 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -70,14 +70,17 @@ pub use updates::Updates; pub struct RecordedUpdates { pub updates: Updates, pub records: usize, + /// Whether `updates` is known to be sorted and consolidated + /// (no duplicate (key, val, time) triples, no zero diffs). + pub consolidated: bool, } impl Default for RecordedUpdates { - fn default() -> Self { Self { updates: Default::default(), records: 0 } } + fn default() -> Self { Self { updates: Default::default(), records: 0, consolidated: true } } } impl Clone for RecordedUpdates { - fn clone(&self) -> Self { Self { updates: self.updates.clone(), records: self.records } } + fn clone(&self) -> Self { Self { updates: self.updates.clone(), records: self.records, consolidated: self.consolidated } } } impl timely::Accountable for RecordedUpdates { @@ -133,7 +136,9 @@ mod container_impls { let t2 = T2::to_inner(t1_owned.clone()); new_times.push(&t2); } + // TODO: Assumes Enter (to_inner) is order-preserving on times. RecordedUpdates { + consolidated: self.consolidated, updates: Updates { keys: self.updates.keys, vals: self.updates.vals, @@ -168,6 +173,7 @@ mod container_impls { RecordedUpdates { updates: flat.consolidate(), records: self.records, + consolidated: true, } } } @@ -185,7 +191,9 @@ mod container_impls { output.push((k, v, &new_time, d)); } } - RecordedUpdates { updates: output, records: self.records } + // TODO: Time advancement may not be order preserving, but .. it could be. + // TODO: Before this is consolidated the above would need to be `form`ed. + RecordedUpdates { updates: output, records: self.records, consolidated: false } } } } @@ -222,7 +230,8 @@ mod column_builder { let mut refs = self.current.borrow().into_index_iter().collect::>(); refs.sort(); let updates = Updates::form(refs.into_iter()); - self.pending.push_back(RecordedUpdates { updates, records }); + if std::env::var("DDIR_VALIDATE").is_ok() { updates.validate("ValColBuilder::push_into"); } + self.pending.push_back(RecordedUpdates { updates, records, consolidated: true }); self.current.clear(); } } @@ -260,7 +269,8 @@ mod column_builder { let mut refs = self.current.borrow().into_index_iter().collect::>(); refs.sort(); let updates = Updates::form(refs.into_iter()); - self.pending.push_back(RecordedUpdates { updates, records }); + if std::env::var("DDIR_VALIDATE").is_ok() { updates.validate("ValColBuilder::finish"); } + self.pending.push_back(RecordedUpdates { updates, records, consolidated: true }); self.current.clear(); } self.empty = self.pending.pop_front(); @@ -327,7 +337,7 @@ mod distributor { let mut first_records = total_records.saturating_sub(non_empty.saturating_sub(1)); for (pusher, output) in pushers.iter_mut().zip(outputs) { if !output.keys.values.is_empty() { - let recorded = RecordedUpdates { updates: output, records: first_records }; + let recorded = RecordedUpdates { updates: output, records: first_records, consolidated: container.consolidated }; first_records = 1; let mut recorded = recorded; Message::push_at(&mut recorded, time.clone(), pusher); @@ -499,150 +509,21 @@ pub mod arrangement { type TimeOwned = U::Time; - fn len(&self) -> usize { self.diffs.values.len() } + fn len(&self) -> usize { self.keys.values.len() } fn clear(&mut self) { *self = Self::default(); } #[inline(never)] fn merge_from(&mut self, others: &mut [Self], positions: &mut [usize]) { - match others.len() { - 0 => {}, - 1 => { - // Bulk copy: take remaining keys from position onward. - let other = &mut others[0]; - let pos = &mut positions[0]; - if self.keys.values.len() == 0 && *pos == 0 { - std::mem::swap(self, other); - return; - } - let other_len = other.keys.values.len(); - self.extend_from_keys(other, *pos .. other_len); - *pos = other_len; - }, - 2 => { - let mut this_sum = U::Diff::default(); - let mut that_sum = U::Diff::default(); - - let (left, right) = others.split_at_mut(1); - let this = &left[0]; - let that = &right[0]; - let this_keys = this.keys.values.borrow(); - let that_keys = that.keys.values.borrow(); - let mut this_key_range = positions[0] .. this_keys.len(); - let mut that_key_range = positions[1] .. that_keys.len(); - - while !this_key_range.is_empty() && !that_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { - let this_key = this_keys.get(this_key_range.start); - let that_key = that_keys.get(that_key_range.start); - match this_key.cmp(&that_key) { - std::cmp::Ordering::Less => { - let lower = this_key_range.start; - gallop(this_keys, &mut this_key_range, |x| x < that_key); - self.extend_from_keys(this, lower .. this_key_range.start); - }, - std::cmp::Ordering::Equal => { - let values_len = self.vals.values.len(); - let mut this_val_range = this.vals_bounds(this_key_range.start .. this_key_range.start+1); - let mut that_val_range = that.vals_bounds(that_key_range.start .. that_key_range.start+1); - while !this_val_range.is_empty() && !that_val_range.is_empty() { - let this_val = this.vals.values.borrow().get(this_val_range.start); - let that_val = that.vals.values.borrow().get(that_val_range.start); - match this_val.cmp(&that_val) { - std::cmp::Ordering::Less => { - let lower = this_val_range.start; - gallop(this.vals.values.borrow(), &mut this_val_range, |x| x < that_val); - self.extend_from_vals(this, lower .. this_val_range.start); - }, - std::cmp::Ordering::Equal => { - let updates_len = self.times.values.len(); - let mut this_time_range = this.times_bounds(this_val_range.start .. this_val_range.start+1); - let mut that_time_range = that.times_bounds(that_val_range.start .. that_val_range.start+1); - while !this_time_range.is_empty() && !that_time_range.is_empty() { - let this_time = this.times.values.borrow().get(this_time_range.start); - let this_diff = this.diffs.values.borrow().get(this_time_range.start); - let that_time = that.times.values.borrow().get(that_time_range.start); - let that_diff = that.diffs.values.borrow().get(that_time_range.start); - match this_time.cmp(&that_time) { - std::cmp::Ordering::Less => { - let lower = this_time_range.start; - gallop(this.times.values.borrow(), &mut this_time_range, |x| x < that_time); - self.times.values.extend_from_self(this.times.values.borrow(), lower .. this_time_range.start); - self.diffs.extend_from_self(this.diffs.borrow(), lower .. this_time_range.start); - }, - std::cmp::Ordering::Equal => { - this_sum.copy_from(this_diff); - that_sum.copy_from(that_diff); - this_sum.plus_equals(&that_sum); - if !this_sum.is_zero() { - self.times.values.push(this_time); - self.diffs.values.push(&this_sum); - self.diffs.bounds.push(self.diffs.values.len() as u64); - } - this_time_range.start += 1; - that_time_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_time_range.start; - gallop(that.times.values.borrow(), &mut that_time_range, |x| x < this_time); - self.times.values.extend_from_self(that.times.values.borrow(), lower .. that_time_range.start); - self.diffs.extend_from_self(that.diffs.borrow(), lower .. that_time_range.start); - }, - } - } - // Remaining from this side - if !this_time_range.is_empty() { - self.times.values.extend_from_self(this.times.values.borrow(), this_time_range.clone()); - self.diffs.extend_from_self(this.diffs.borrow(), this_time_range.clone()); - } - // Remaining from that side - if !that_time_range.is_empty() { - self.times.values.extend_from_self(that.times.values.borrow(), that_time_range.clone()); - self.diffs.extend_from_self(that.diffs.borrow(), that_time_range.clone()); - } - if self.times.values.len() > updates_len { - self.times.bounds.push(self.times.values.len() as u64); - self.vals.values.push(this_val); - } - this_val_range.start += 1; - that_val_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_val_range.start; - gallop(that.vals.values.borrow(), &mut that_val_range, |x| x < this_val); - self.extend_from_vals(that, lower .. that_val_range.start); - }, - } - } - self.extend_from_vals(this, this_val_range); - self.extend_from_vals(that, that_val_range); - if self.vals.values.len() > values_len { - self.vals.bounds.push(self.vals.values.len() as u64); - self.keys.values.push(this_key); - } - this_key_range.start += 1; - that_key_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_key_range.start; - gallop(that_keys, &mut that_key_range, |x| x < this_key); - self.extend_from_keys(that, lower .. that_key_range.start); - }, - } - } - // Copy remaining from whichever side has data, up to capacity. - while !this_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { - let lower = this_key_range.start; - this_key_range.start = this_key_range.end; // take all remaining - self.extend_from_keys(this, lower .. this_key_range.start); - } - while !that_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { - let lower = that_key_range.start; - that_key_range.start = that_key_range.end; - self.extend_from_keys(that, lower .. that_key_range.start); - } - positions[0] = this_key_range.start; - positions[1] = that_key_range.start; - }, - n => unimplemented!("{n}-way merge not supported"), + // Slow but correct: collect self + all others (skipping already-consumed updates), + // rebuild via form_unsorted which sorts and consolidates. + let self_iter = self.iter(); + let others_iter = others.iter().zip(positions.iter()).flat_map(|(other, pos)| { + other.iter().skip(*pos) + }); + *self = Self::form_unsorted(self_iter.chain(others_iter)); + // Mark all inputs as fully consumed. + for (other, pos) in others.iter().zip(positions.iter_mut()) { + *pos = other.len(); } } @@ -653,49 +534,35 @@ pub mod arrangement { keep: &mut Self, ship: &mut Self, ) { + use columnar::Columnar; + // Slow but correct: partition, form_unsorted onto keep/ship. + let mut keep_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); + let mut ship_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); + let mut time = U::Time::default(); - for key_idx in 0 .. self.keys.values.len() { - let key = self.keys.values.borrow().get(key_idx); - let keep_vals_len = keep.vals.values.len(); - let ship_vals_len = ship.vals.values.len(); - for val_idx in self.vals_bounds(key_idx..key_idx+1) { - let val = self.vals.values.borrow().get(val_idx); - let keep_times_len = keep.times.values.len(); - let ship_times_len = ship.times.values.len(); - for time_idx in self.times_bounds(val_idx..val_idx+1) { - let t = self.times.values.borrow().get(time_idx); - let diff = self.diffs.values.borrow().get(time_idx); - time.copy_from(t); - if upper.less_equal(&time) { - frontier.insert_ref(&time); - keep.times.values.push(t); - keep.diffs.values.push(diff); - keep.diffs.bounds.push(keep.diffs.values.len() as u64); - } - else { - ship.times.values.push(t); - ship.diffs.values.push(diff); - ship.diffs.bounds.push(ship.diffs.values.len() as u64); - } - } - if keep.times.values.len() > keep_times_len { - keep.times.bounds.push(keep.times.values.len() as u64); - keep.vals.values.push(val); - } - if ship.times.values.len() > ship_times_len { - ship.times.bounds.push(ship.times.values.len() as u64); - ship.vals.values.push(val); - } - } - if keep.vals.values.len() > keep_vals_len { - keep.vals.bounds.push(keep.vals.values.len() as u64); - keep.keys.values.push(key); - } - if ship.vals.values.len() > ship_vals_len { - ship.vals.bounds.push(ship.vals.values.len() as u64); - ship.keys.values.push(key); + for (k, v, t, d) in self.iter() { + Columnar::copy_from(&mut time, t); + if upper.less_equal(&time) { + frontier.insert_ref(&time); + keep_vec.push((Columnar::into_owned(k), Columnar::into_owned(v), time.clone(), Columnar::into_owned(d))); + } else { + ship_vec.push((Columnar::into_owned(k), Columnar::into_owned(v), time.clone(), Columnar::into_owned(d))); } } + + // Merge new data into keep/ship: push existing + new into flat Updates, consolidate. + if !keep_vec.is_empty() { + let mut flat = Self::default(); + for (k, v, t, d) in keep.iter() { flat.push((k, v, t, d)); } + for (k, v, t, d) in keep_vec.iter() { flat.push((k, v, t, d)); } + *keep = flat.consolidate(); + } + if !ship_vec.is_empty() { + let mut flat = Self::default(); + for (k, v, t, d) in ship.iter() { flat.push((k, v, t, d)); } + for (k, v, t, d) in ship_vec.iter() { flat.push((k, v, t, d)); } + *ship = flat.consolidate(); + } } } @@ -748,7 +615,7 @@ pub mod arrangement { } pub struct ValMirror { - current: Updates, + chunks: Vec>, } impl differential_dataflow::trace::Builder for ValMirror { type Time = U::Time; @@ -756,64 +623,49 @@ pub mod arrangement { type Output = OrdValBatch>; fn with_capacity(_keys: usize, _vals: usize, _upds: usize) -> Self { - Self { current: Updates::default() } + Self { chunks: Vec::new() } } fn push(&mut self, chunk: &mut Self::Input) { - use columnar::Len; - let len = chunk.keys.values.len(); - if len > 0 { - self.current.extend_from_keys(chunk, 0..len); + if chunk.len() > 0 { + self.chunks.push(std::mem::take(chunk)); } } fn done(self, description: Description) -> Self::Output { - let mut chain = if self.current.len() > 0 { - vec![self.current] - } else { - vec![] - }; + let mut chain = self.chunks; Self::seal(&mut chain, description) } fn seal(chain: &mut Vec, description: Description) -> Self::Output { - if chain.len() == 0 { + use columnar::Len; + + // Slow but correct: merge all chain entries via form_unsorted. + let merged: Updates = Updates::form_unsorted(chain.iter().flat_map(|c| c.iter())); + chain.clear(); + + let updates = Len::len(&merged.diffs.values); + if updates == 0 { let storage = OrdValStorage { keys: Default::default(), vals: Default::default(), upds: Default::default(), }; OrdValBatch { storage, description, updates: 0 } - } - else if chain.len() == 1 { - use columnar::Len; - let storage = chain.pop().unwrap(); - let updates = storage.diffs.values.len(); - let val_offs = strides_to_offset_list(&storage.vals.bounds, storage.keys.values.len()); - let time_offs = strides_to_offset_list(&storage.times.bounds, storage.vals.values.len()); + } else { + let val_offs = strides_to_offset_list(&merged.vals.bounds, Len::len(&merged.keys.values)); + let time_offs = strides_to_offset_list(&merged.times.bounds, Len::len(&merged.vals.values)); let storage = OrdValStorage { - keys: Coltainer { container: storage.keys.values }, + keys: Coltainer { container: merged.keys.values }, vals: Vals { offs: val_offs, - vals: Coltainer { container: storage.vals.values }, + vals: Coltainer { container: merged.vals.values }, }, upds: Upds { offs: time_offs, - times: Coltainer { container: storage.times.values }, - diffs: Coltainer { container: storage.diffs.values }, + times: Coltainer { container: merged.times.values }, + diffs: Coltainer { container: merged.diffs.values }, }, }; OrdValBatch { storage, description, updates } } - else { - use columnar::Len; - let mut merged = chain.remove(0); - for other in chain.drain(..) { - let len = other.keys.values.len(); - if len > 0 { - merged.extend_from_keys(&other, 0..len); - } - } - chain.push(merged); - Self::seal(chain, description) - } } } @@ -888,6 +740,57 @@ pub mod updates { lower..upper } + /// A streaming consolidation iterator for sorted `(key, val, time, diff)` data. + /// + /// Accumulates diffs for equal `(key, val, time)` triples, yielding at most + /// one output per distinct triple, with a non-zero accumulated diff. + /// Input must be sorted by `(key, val, time)`. + pub struct Consolidating { + iter: std::iter::Peekable, + diff: D, + } + + impl Consolidating + where + K: Copy + Eq, + V: Copy + Eq, + T: Copy + Eq, + D: Semigroup + IsZero + Default, + I: Iterator, + { + pub fn new(iter: I) -> Self { + Self { iter: iter.peekable(), diff: D::default() } + } + } + + impl Iterator for Consolidating + where + K: Copy + Eq, + V: Copy + Eq, + T: Copy + Eq, + D: Semigroup + IsZero + Default + Clone, + I: Iterator, + { + type Item = (K, V, T, D); + fn next(&mut self) -> Option { + loop { + let (k, v, t, d) = self.iter.next()?; + self.diff = d; + while let Some(&(k2, v2, t2, _)) = self.iter.peek() { + if k2 == k && v2 == v && t2 == t { + let (_, _, _, d2) = self.iter.next().unwrap(); + self.diff.plus_equals(&d2); + } else { + break; + } + } + if !self.diff.is_zero() { + return Some((k, v, t, self.diff.clone())); + } + } + } + } + impl Updates { pub fn vals_bounds(&self, key_range: std::ops::Range) -> std::ops::Range { @@ -933,98 +836,64 @@ pub mod updates { self.diffs.extend_from_self(other.diffs.borrow(), time_range); } - /// Forms a consolidated `Updates` from sorted `(key, val, time, diff)` refs. - /// - /// Tracks a `prev` reference to the previous element. On each new element, - /// compares against `prev` to detect key/val/time changes. Only pushes - /// accumulated diffs when they are nonzero, and only emits times/vals/keys - /// that have at least one nonzero diff beneath them. - pub fn form<'a>(mut sorted: impl Iterator>>) -> Self { + /// Forms a consolidated `Updates` trie from unsorted `(key, val, time, diff)` refs. + pub fn form_unsorted<'a>(unsorted: impl Iterator>>) -> Self { + let mut data = unsorted.collect::>(); + data.sort(); + Self::form(data.into_iter()) + } + + /// Forms a consolidated `Updates` trie from sorted `(key, val, time, diff)` refs. + pub fn form<'a>(sorted: impl Iterator>>) -> Self { + // Step 1: Streaming consolidation — accumulate diffs, drop zeros. + let consolidated = Consolidating::new( + sorted.map(|(k, v, t, d)| (k, v, t, ::into_owned(d))) + ); + + // Step 2: Build the trie from consolidated, sorted, non-zero data. let mut output = Self::default(); - let mut diff_stash = U::Diff::default(); - let mut diff_temp = U::Diff::default(); - - if let Some(first) = sorted.next() { - - let mut prev = first; - Columnar::copy_from(&mut diff_stash, prev.3); - - for curr in sorted { - let key_differs = ContainerOf::::reborrow_ref(curr.0) != ContainerOf::::reborrow_ref(prev.0); - let val_differs = key_differs || ContainerOf::::reborrow_ref(curr.1) != ContainerOf::::reborrow_ref(prev.1); - let time_differs = val_differs || ContainerOf::::reborrow_ref(curr.2) != ContainerOf::::reborrow_ref(prev.2); - - if time_differs { - // Flush the accumulated diff for prev's (key, val, time). - if !diff_stash.is_zero() { - // We have a real update to emit. Push time (and val/key - // if this is the first time under them). - let times_len = output.times.values.len(); - let vals_len = output.vals.values.len(); - - if val_differs { - // Seal the previous val's time list, if any times were emitted. - if times_len > 0 { - output.times.bounds.push(times_len as u64); - } - if key_differs { - // Seal the previous key's val list, if any vals were emitted. - if vals_len > 0 { - output.vals.bounds.push(vals_len as u64); - } - output.keys.values.push(prev.0); - } - output.vals.values.push(prev.1); - } - output.times.values.push(prev.2); - output.diffs.values.push(&diff_stash); - output.diffs.bounds.push(output.diffs.values.len() as u64); - } - Columnar::copy_from(&mut diff_stash, curr.3); - } else { - // Same (key, val, time): accumulate diff. - Columnar::copy_from(&mut diff_temp, curr.3); - diff_stash.plus_equals(&diff_temp); + let mut updates = consolidated; + if let Some((key, val, time, diff)) = updates.next() { + let mut prev = (key, val, time); + output.keys.values.push(key); + output.vals.values.push(val); + output.times.values.push(time); + output.diffs.values.push(&diff); + output.diffs.bounds.push(output.diffs.values.len() as u64); + + // As we proceed, seal up known complete runs. + for (key, val, time, diff) in updates { + + // If keys differ, record key and seal vals and times. + if key != prev.0 { + output.vals.bounds.push(output.vals.values.len() as u64); + output.times.bounds.push(output.times.values.len() as u64); + output.keys.values.push(key); + output.vals.values.push(val); } - prev = curr; - } - - // Flush the final accumulated diff. - if !diff_stash.is_zero() { - let keys_len = output.keys.values.len(); - let vals_len = output.vals.values.len(); - let times_len = output.times.values.len(); - let need_key = keys_len == 0 || ContainerOf::::reborrow_ref(prev.0) != output.keys.values.borrow().get(keys_len - 1); - let need_val = need_key || vals_len == 0 || ContainerOf::::reborrow_ref(prev.1) != output.vals.values.borrow().get(vals_len - 1); - - if need_val { - if times_len > 0 { - output.times.bounds.push(times_len as u64); - } - if need_key { - if vals_len > 0 { - output.vals.bounds.push(vals_len as u64); - } - output.keys.values.push(prev.0); - } - output.vals.values.push(prev.1); + // If vals differ, record val and seal times. + else if val != prev.1 { + output.times.bounds.push(output.times.values.len() as u64); + output.vals.values.push(val); } - output.times.values.push(prev.2); - output.diffs.values.push(&diff_stash); + else { + // We better not find a duplicate time. + assert!(time != prev.2); + } + + // Always record (time, diff). + output.times.values.push(time); + output.diffs.values.push(&diff); output.diffs.bounds.push(output.diffs.values.len() as u64); - } - // Seal the final groups at each level. - if !output.times.values.is_empty() { - output.times.bounds.push(output.times.values.len() as u64); - } - if !output.vals.values.is_empty() { - output.vals.bounds.push(output.vals.values.len() as u64); - } - if !output.keys.values.is_empty() { - output.keys.bounds.push(output.keys.values.len() as u64); + prev = (key, val, time); } + + // Seal up open lists. + output.keys.bounds.push(output.keys.values.len() as u64); + output.vals.bounds.push(output.vals.values.len() as u64); + output.times.bounds.push(output.times.values.len() as u64); } output @@ -1034,102 +903,60 @@ pub mod updates { /// single outer key list, all lists sorted and deduplicated, /// diff lists are singletons (or absent if cancelled). pub fn consolidate(self) -> Self { + Self::form_unsorted(self.iter()) + } - let Self { keys, vals, times, diffs } = self; - - let keys_b = keys.borrow(); - let vals_b = vals.borrow(); - let times_b = times.borrow(); - let diffs_b = diffs.borrow(); - - // Flatten to index tuples: [key_abs, val_abs, time_abs, diff_abs]. - let mut tuples: Vec<[usize; 4]> = Vec::new(); - for outer in 0..Len::len(&keys_b) { - for k in child_range(keys_b.bounds, outer) { - for v in child_range(vals_b.bounds, k) { - for t in child_range(times_b.bounds, v) { - for d in child_range(diffs_b.bounds, t) { - tuples.push([k, v, t, d]); - } - } - } - } - } - - // Sort by (key, val, time). Diff is payload. - tuples.sort_by(|a, b| { - keys_b.values.get(a[0]).cmp(&keys_b.values.get(b[0])) - .then_with(|| vals_b.values.get(a[1]).cmp(&vals_b.values.get(b[1]))) - .then_with(|| times_b.values.get(a[2]).cmp(×_b.values.get(b[2]))) - }); - - // Build consolidated output, bottom-up cancellation. - let mut output = Self::default(); - let mut diff_stash = U::Diff::default(); - let mut diff_temp = U::Diff::default(); - - let mut idx = 0; - while idx < tuples.len() { - let key_ref = keys_b.values.get(tuples[idx][0]); - let key_start_vals = output.vals.values.len(); - - // All entries with this key. - while idx < tuples.len() && keys_b.values.get(tuples[idx][0]) == key_ref { - let val_ref = vals_b.values.get(tuples[idx][1]); - let val_start_times = output.times.values.len(); - - // All entries with this (key, val). - while idx < tuples.len() - && keys_b.values.get(tuples[idx][0]) == key_ref - && vals_b.values.get(tuples[idx][1]) == val_ref - { - let time_ref = times_b.values.get(tuples[idx][2]); - - // Sum all diffs for this (key, val, time). - Columnar::copy_from(&mut diff_stash, diffs_b.values.get(tuples[idx][3])); - idx += 1; - while idx < tuples.len() - && keys_b.values.get(tuples[idx][0]) == key_ref - && vals_b.values.get(tuples[idx][1]) == val_ref - && times_b.values.get(tuples[idx][2]) == time_ref - { - Columnar::copy_from(&mut diff_temp, diffs_b.values.get(tuples[idx][3])); - diff_stash.plus_equals(&diff_temp); - idx += 1; - } - - // Emit time + singleton diff if nonzero. - if !diff_stash.is_zero() { - output.times.values.push(time_ref); - output.diffs.values.push(&diff_stash); - output.diffs.bounds.push(output.diffs.values.len() as u64); - } - } - - // Seal time list for this val; emit val if any times survived. - if output.times.values.len() > val_start_times { - output.times.bounds.push(output.times.values.len() as u64); - output.vals.values.push(val_ref); - } - } + /// The number of leaf-level diff entries (total updates). + pub fn len(&self) -> usize { self.diffs.values.len() } - // Seal val list for this key; emit key if any vals survived. - if output.vals.values.len() > key_start_vals { - output.vals.bounds.push(output.vals.values.len() as u64); - output.keys.values.push(key_ref); + /// Checks trie invariants and panics with a descriptive message if any fail. + pub fn validate(&self, label: &str) { + let keys_len = Len::len(&self.keys.values); + let vals_bounds_len = Len::len(&self.vals.bounds); + let vals_len = Len::len(&self.vals.values); + let times_bounds_len = Len::len(&self.times.bounds); + let times_len = Len::len(&self.times.values); + let diffs_bounds_len = Len::len(&self.diffs.bounds); + let diffs_len = Len::len(&self.diffs.values); + + // 1. One val-group bound per key. + assert_eq!(vals_bounds_len, keys_len, + "{label}: vals.bounds.len() ({vals_bounds_len}) != keys.values.len() ({keys_len})"); + + // 2. One time-group bound per val. + assert_eq!(times_bounds_len, vals_len, + "{label}: times.bounds.len() ({times_bounds_len}) != vals.values.len() ({vals_len})"); + + // 3. One diff-group bound per time (singleton diffs). + assert_eq!(diffs_bounds_len, times_len, + "{label}: diffs.bounds.len() ({diffs_bounds_len}) != times.values.len() ({times_len})"); + + // 4. Cumulative bounds are monotonically non-decreasing at each level. + // 5. Final bound at each level matches the child values count. + fn check_monotone_and_final + Len>( + bounds: &B, + child_len: usize, + level: &str, + label: &str, + ) { + let n = Len::len(bounds); + if n == 0 { return; } + let mut prev = bounds.index_as(0); + for i in 1..n { + let curr = bounds.index_as(i); + assert!(curr >= prev, + "{label}: {level}.bounds not monotonic at index {i}: {prev} > {curr}"); + prev = curr; } + let last = bounds.index_as(n - 1) as usize; + assert_eq!(last, child_len, + "{label}: last {level}.bounds ({last}) != child values len ({child_len})"); } - // Seal the single outer key list. - if !output.keys.values.is_empty() { - output.keys.bounds.push(output.keys.values.len() as u64); - } - - output + check_monotone_and_final(&self.vals.bounds.borrow(), vals_len, "vals", label); + check_monotone_and_final(&self.times.bounds.borrow(), times_len, "times", label); + check_monotone_and_final(&self.diffs.bounds.borrow(), diffs_len, "diffs", label); } - - /// The number of leaf-level diff entries (total updates). - pub fn len(&self) -> usize { self.diffs.values.len() } } /// Push a single flat update as a stride-1 entry. @@ -1203,6 +1030,54 @@ pub mod updates { fn into_bytes(&self, _writer: &mut W) { unimplemented!() } } + /// An incremental trie builder that accepts sorted, consolidated `Updates` chunks + /// and melds them into a single `Updates` trie. + /// + /// The internal `Updates` has open (unsealed) bounds at the keys, vals, and times + /// levels — the last group at each level has its values pushed but no corresponding + /// bounds entry. `diffs.bounds` is always 1:1 with `times.values`. + /// + /// `meld` accepts a consolidated `Updates` whose first `(key, val, time)` is + /// strictly greater than the builder's last `(key, val, time)`. The key and val + /// may equal the builder's current open key/val, as long as the time is greater. + /// + /// `done` seals all open bounds and returns the completed `Updates`. + pub struct UpdatesBuilder { + /// Non-empty, consolidated updates. + updates: Updates, + } + + impl UpdatesBuilder { + /// Construct a new builder from consolidated updates. + /// + /// If the updates are not consolidated none of this works. + pub fn new_from(updates: Updates) -> Self { Self { updates } } + + /// Meld a sorted, consolidated `Updates` chunk into this builder. + /// + /// The chunk's first `(key, val, time)` must be strictly greater than + /// the builder's last `(key, val, time)`. Keys and vals may overlap + /// (continue the current group), but times must be strictly increasing + /// within the same `(key, val)`. + pub fn meld(&mut self, chunk: &Updates) { + todo!("to be implemented") + } + + /// Seal all open bounds and return the completed `Updates`. + pub fn done(mut self) -> Updates { + use columnar::Len; + if Len::len(&self.updates.keys.values) > 0 { + // Seal the open time group. + self.updates.times.bounds.push(Len::len(&self.updates.times.values) as u64); + // Seal the open val group. + self.updates.vals.bounds.push(Len::len(&self.updates.vals.values) as u64); + // Seal the outer key group. + self.updates.keys.bounds.push(Len::len(&self.updates.keys.values) as u64); + } + self.updates + } + } + #[cfg(test)] mod tests { use super::*; @@ -1348,6 +1223,9 @@ where .unary::, _, _, _>(Pipeline, "JoinFunction", move |_, _| { move |input, output| { input.for_each(|time, data| { + if std::env::var("DDIR_VALIDATE").is_ok() { + data.updates.validate("JoinFunction input"); + } let mut session = output.session_with_builder(&time); for (k1, v1, t1, d1) in data.updates.iter() { let t1o: U::Time = Columnar::into_owned(t1); From b25fd430c0b709e84efeb352ff53d1c07332c0ed Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 26 Mar 2026 06:21:08 -0400 Subject: [PATCH 02/27] WIP --- .../examples/columnar/columnar_support.rs | 317 +++++++++++++++--- differential-dataflow/src/operators/reduce.rs | 2 + 2 files changed, 279 insertions(+), 40 deletions(-) diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index b6a67af25..c4f17de87 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -443,8 +443,8 @@ pub mod arrangement { } use crate::{Updates, RecordedUpdates}; - use differential_dataflow::trace::implementations::merge_batcher::{MergeBatcher, InternalMerger}; - type ValBatcher2 = MergeBatcher, TrieChunker, InternalMerger>>; + use differential_dataflow::trace::implementations::merge_batcher::MergeBatcher; + type ValBatcher2 = MergeBatcher, TrieChunker, trie_merger::TrieMerger>; /// A chunker that unwraps `RecordedUpdates` into bare `Updates` for the merge batcher. /// The `records` accounting is discarded here — it has served its purpose for exchange. @@ -488,8 +488,7 @@ pub mod arrangement { pub mod batcher { - use std::ops::Range; - use columnar::{Borrow, Columnar, Container, Index, Len, Push}; + use columnar::{Borrow, Columnar, Index, Len, Push}; use differential_dataflow::difference::{Semigroup, IsZero}; use timely::progress::frontier::{Antichain, AntichainRef}; use differential_dataflow::trace::implementations::merge_batcher::container::InternalMerge; @@ -499,34 +498,34 @@ pub mod arrangement { impl timely::container::SizableContainer for Updates { fn at_capacity(&self) -> bool { - use columnar::Len; self.diffs.values.len() >= 64 * 1024 } fn ensure_capacity(&mut self, _stash: &mut Option) { } } + /// Required by `reduce_abelian`'s bound `Builder::Input: InternalMerge`. + /// Not called at runtime — our batcher uses `TrieMerger` instead. + /// TODO: Relax the bound in DD's reduce to remove this requirement. impl InternalMerge for Updates { - type TimeOwned = U::Time; - - fn len(&self) -> usize { self.keys.values.len() } - fn clear(&mut self) { *self = Self::default(); } - - #[inline(never)] + fn len(&self) -> usize { Len::len(&self.keys.values) } + fn clear(&mut self) { + use columnar::Clear; + self.keys.clear(); + self.vals.clear(); + self.times.clear(); + self.diffs.clear(); + } fn merge_from(&mut self, others: &mut [Self], positions: &mut [usize]) { - // Slow but correct: collect self + all others (skipping already-consumed updates), - // rebuild via form_unsorted which sorts and consolidates. let self_iter = self.iter(); let others_iter = others.iter().zip(positions.iter()).flat_map(|(other, pos)| { other.iter().skip(*pos) }); *self = Self::form_unsorted(self_iter.chain(others_iter)); - // Mark all inputs as fully consumed. for (other, pos) in others.iter().zip(positions.iter_mut()) { *pos = other.len(); } } - fn extract( &mut self, upper: AntichainRef, @@ -534,12 +533,9 @@ pub mod arrangement { keep: &mut Self, ship: &mut Self, ) { - use columnar::Columnar; - // Slow but correct: partition, form_unsorted onto keep/ship. + let mut time = U::Time::default(); let mut keep_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); let mut ship_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); - - let mut time = U::Time::default(); for (k, v, t, d) in self.iter() { Columnar::copy_from(&mut time, t); if upper.less_equal(&time) { @@ -549,8 +545,6 @@ pub mod arrangement { ship_vec.push((Columnar::into_owned(k), Columnar::into_owned(v), time.clone(), Columnar::into_owned(d))); } } - - // Merge new data into keep/ship: push existing + new into flat Updates, consolidate. if !keep_vec.is_empty() { let mut flat = Self::default(); for (k, v, t, d) in keep.iter() { flat.push((k, v, t, d)); } @@ -565,28 +559,135 @@ pub mod arrangement { } } } + } + + pub mod trie_merger { + + use columnar::{Columnar, Len}; + use timely::PartialOrder; + use timely::progress::frontier::{Antichain, AntichainRef}; + use differential_dataflow::trace::implementations::merge_batcher::Merger; + + use crate::ColumnarUpdate as Update; + use crate::Updates; - #[inline(always)] - pub(crate) fn gallop(input: TC, range: &mut Range, mut cmp: impl FnMut(::Ref) -> bool) { - // if empty input, or already >= element, return - if !Range::::is_empty(range) && cmp(input.get(range.start)) { - let mut step = 1; - while range.start + step < range.end && cmp(input.get(range.start + step)) { - range.start += step; - step <<= 1; + pub struct TrieMerger { + _marker: std::marker::PhantomData, + } + + impl Default for TrieMerger { + fn default() -> Self { Self { _marker: std::marker::PhantomData } } + } + + /// A merging iterator over two sorted iterators. + struct Merging { + iter1: std::iter::Peekable, + iter2: std::iter::Peekable, + } + + impl Iterator for Merging + where + K: Copy + Ord, + V: Copy + Ord, + T: Copy + Ord, + I1: Iterator, + I2: Iterator, + { + type Item = (K, V, T, D); + #[inline] + fn next(&mut self) -> Option { + match (self.iter1.peek(), self.iter2.peek()) { + (Some(a), Some(b)) => { + if (a.0, a.1, a.2) <= (b.0, b.1, b.2) { + self.iter1.next() + } else { + self.iter2.next() + } + } + (Some(_), None) => self.iter1.next(), + (None, Some(_)) => self.iter2.next(), + (None, None) => None, + } + } + } + + /// Build sorted `Updates` chunks from a sorted iterator of refs, + /// using `Updates::form` (which consolidates internally) on batches. + fn form_chunks<'a, U: Update>( + sorted: impl Iterator>>, + output: &mut Vec>, + ) { + let mut sorted = sorted.peekable(); + while sorted.peek().is_some() { + let chunk = Updates::::form((&mut sorted).take(64 * 1024)); + if chunk.len() > 0 { + output.push(chunk); } + } + } + + impl Merger for TrieMerger + where + U::Time: Ord + PartialOrder + Clone + 'static, + { + type Chunk = Updates; + type Time = U::Time; + + fn merge( + &mut self, + list1: Vec>, + list2: Vec>, + output: &mut Vec>, + _stash: &mut Vec>, + ) { + let iter1 = list1.iter().flat_map(|chunk| chunk.iter()); + let iter2 = list2.iter().flat_map(|chunk| chunk.iter()); + + let merged = Merging { + iter1: iter1.peekable(), + iter2: iter2.peekable(), + }; + + form_chunks::(merged, output); + } - step >>= 1; - while step > 0 { - if range.start + step < range.end && cmp(input.get(range.start + step)) { - range.start += step; + fn extract( + &mut self, + merged: Vec, + upper: AntichainRef, + frontier: &mut Antichain, + ship: &mut Vec, + kept: &mut Vec, + _stash: &mut Vec, + ) { + // Flatten the sorted, consolidated chain into refs. + let all = merged.iter().flat_map(|chunk| chunk.iter()); + + // Partition into two sorted streams by time. + let mut time_owned = U::Time::default(); + let mut keep_vec = Vec::new(); + let mut ship_vec = Vec::new(); + for (k, v, t, d) in all { + Columnar::copy_from(&mut time_owned, t); + if upper.less_equal(&time_owned) { + frontier.insert_ref(&time_owned); + keep_vec.push((k, v, t, d)); + } else { + ship_vec.push((k, v, t, d)); } - step >>= 1; } - range.start += 1; + // Build chunks via form (which consolidates internally). + form_chunks::(keep_vec.into_iter(), kept); + form_chunks::(ship_vec.into_iter(), ship); + } + + fn account(chunk: &Self::Chunk) -> (usize, usize, usize, usize) { + use timely::Accountable; + (chunk.record_count() as usize, 0, 0, 0) } } + } use builder::ValMirror; @@ -637,8 +738,19 @@ pub mod arrangement { fn seal(chain: &mut Vec, description: Description) -> Self::Output { use columnar::Len; - // Slow but correct: merge all chain entries via form_unsorted. - let merged: Updates = Updates::form_unsorted(chain.iter().flat_map(|c| c.iter())); + // Meld sorted, consolidated chain entries in order. + // Pre-allocate to avoid reallocations during meld. + use columnar::{Borrow, Container}; + let mut updates = Updates::::default(); + updates.keys.reserve_for(chain.iter().map(|c| c.keys.borrow())); + updates.vals.reserve_for(chain.iter().map(|c| c.vals.borrow())); + updates.times.reserve_for(chain.iter().map(|c| c.times.borrow())); + updates.diffs.reserve_for(chain.iter().map(|c| c.diffs.borrow())); + let mut builder = crate::updates::UpdatesBuilder::new_from(updates); + for chunk in chain.iter() { + builder.meld(chunk); + } + let merged = builder.done(); chain.clear(); let updates = Len::len(&merged.diffs.values); @@ -1048,10 +1160,20 @@ pub mod updates { } impl UpdatesBuilder { - /// Construct a new builder from consolidated updates. + /// Construct a new builder from consolidated, sealed updates. /// + /// Unseals the last group at keys, vals, and times levels so that + /// subsequent `meld` calls can extend the open groups. /// If the updates are not consolidated none of this works. - pub fn new_from(updates: Updates) -> Self { Self { updates } } + pub fn new_from(mut updates: Updates) -> Self { + use columnar::Len; + if Len::len(&updates.keys.values) > 0 { + updates.keys.bounds.pop(); + updates.vals.bounds.pop(); + updates.times.bounds.pop(); + } + Self { updates } + } /// Meld a sorted, consolidated `Updates` chunk into this builder. /// @@ -1060,7 +1182,122 @@ pub mod updates { /// (continue the current group), but times must be strictly increasing /// within the same `(key, val)`. pub fn meld(&mut self, chunk: &Updates) { - todo!("to be implemented") + use columnar::{Borrow, Index, Len}; + + if chunk.len() == 0 { return; } + + // Empty builder: clone the chunk and unseal it. + if Len::len(&self.updates.keys.values) == 0 { + self.updates = chunk.clone(); + self.updates.keys.bounds.pop(); + self.updates.vals.bounds.pop(); + self.updates.times.bounds.pop(); + return; + } + + // Pre-compute boundary comparisons before mutating. + let keys_match = { + let skb = self.updates.keys.values.borrow(); + let ckb = chunk.keys.values.borrow(); + skb.get(Len::len(&skb) - 1) == ckb.get(0) + }; + let vals_match = keys_match && { + let svb = self.updates.vals.values.borrow(); + let cvb = chunk.vals.values.borrow(); + svb.get(Len::len(&svb) - 1) == cvb.get(0) + }; + + let chunk_num_keys = Len::len(&chunk.keys.values); + let chunk_num_vals = Len::len(&chunk.vals.values); + let chunk_num_times = Len::len(&chunk.times.values); + + // Child ranges for the first element at each level of the chunk. + let first_key_vals = child_range(chunk.vals.borrow().bounds, 0); + let first_val_times = child_range(chunk.times.borrow().bounds, 0); + + // There is a first position where coordinates disagree. + // Strictly beyond that position: seal bounds, extend lists, re-open the last bound. + // At that position: meld the first list, extend subsequent lists, re-open. + let mut differ = false; + + // --- Keys --- + if keys_match { + // Skip the duplicate first key; add remaining keys. + if chunk_num_keys > 1 { + self.updates.keys.values.extend_from_self(chunk.keys.values.borrow(), 1..chunk_num_keys); + } + } else { + // All keys are new. + self.updates.keys.values.extend_from_self(chunk.keys.values.borrow(), 0..chunk_num_keys); + differ = true; + } + + // --- Vals --- + if differ { + // Keys differed: seal open val group, extend all val lists, unseal last. + self.updates.vals.bounds.push(Len::len(&self.updates.vals.values) as u64); + self.updates.vals.extend_from_self(chunk.vals.borrow(), 0..chunk_num_keys); + self.updates.vals.bounds.pop(); + } else { + // Keys matched: meld vals for the shared key. + if vals_match { + // Skip the duplicate first val; add remaining vals from the first key's list. + if first_key_vals.len() > 1 { + self.updates.vals.values.extend_from_self( + chunk.vals.values.borrow(), + (first_key_vals.start + 1)..first_key_vals.end, + ); + } + } else { + // First val differs: add all vals from the first key's list. + self.updates.vals.values.extend_from_self( + chunk.vals.values.borrow(), + first_key_vals.clone(), + ); + differ = true; + } + // Seal the matched key's val group, extend remaining keys' val lists, unseal. + if chunk_num_keys > 1 { + self.updates.vals.bounds.push(Len::len(&self.updates.vals.values) as u64); + self.updates.vals.extend_from_self(chunk.vals.borrow(), 1..chunk_num_keys); + self.updates.vals.bounds.pop(); + } + } + + // --- Times --- + if differ { + // Seal open time group, extend all time lists, unseal last. + self.updates.times.bounds.push(Len::len(&self.updates.times.values) as u64); + self.updates.times.extend_from_self(chunk.times.borrow(), 0..chunk_num_vals); + self.updates.times.bounds.pop(); + } else { + // Keys and vals matched. Times must be strictly greater (precondition), + // so we always set differ = true here. + debug_assert!({ + let stb = self.updates.times.values.borrow(); + let ctb = chunk.times.values.borrow(); + stb.get(Len::len(&stb) - 1) != ctb.get(0) + }, "meld: duplicate time within same (key, val)"); + // Add times from the first val's time list into the open group. + self.updates.times.values.extend_from_self( + chunk.times.values.borrow(), + first_val_times.clone(), + ); + differ = true; + // Seal the matched val's time group, extend remaining vals' time lists, unseal. + if chunk_num_vals > 1 { + self.updates.times.bounds.push(Len::len(&self.updates.times.values) as u64); + self.updates.times.extend_from_self(chunk.times.borrow(), 1..chunk_num_vals); + self.updates.times.bounds.pop(); + } + } + + // --- Diffs --- + // Diffs are always sealed (1:1 with times). By the precondition that + // times are strictly increasing for the same (key, val), differ is + // always true by this point — just extend all diff lists. + debug_assert!(differ); + self.updates.diffs.extend_from_self(chunk.diffs.borrow(), 0..chunk_num_times); } /// Seal all open bounds and return the completed `Updates`. diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index fe3622d74..77834ab42 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -23,6 +23,8 @@ use crate::trace::implementations::containers::BatchContainer; use crate::trace::implementations::merge_batcher::container::InternalMerge; use crate::trace::TraceReader; +// TODO: Remove the InternalMerge constraint on Bu::Input. It only needs Clear. + /// A key-wise reduction of values in an input trace. /// /// This method exists to provide reduce functionality without opinions about qualifying trace types. From a7e92f69e067f00fa6246de202f5b87a001b6707 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 28 Mar 2026 20:33:22 -0400 Subject: [PATCH 03/27] Wildly overcomplicated --- .../examples/columnar/columnar_support.rs | 638 ++++++++++++++---- 1 file changed, 522 insertions(+), 116 deletions(-) diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index c4f17de87..149a93b60 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -230,7 +230,6 @@ mod column_builder { let mut refs = self.current.borrow().into_index_iter().collect::>(); refs.sort(); let updates = Updates::form(refs.into_iter()); - if std::env::var("DDIR_VALIDATE").is_ok() { updates.validate("ValColBuilder::push_into"); } self.pending.push_back(RecordedUpdates { updates, records, consolidated: true }); self.current.clear(); } @@ -269,7 +268,6 @@ mod column_builder { let mut refs = self.current.borrow().into_index_iter().collect::>(); refs.sort(); let updates = Updates::form(refs.into_iter()); - if std::env::var("DDIR_VALIDATE").is_ok() { updates.validate("ValColBuilder::finish"); } self.pending.push_back(RecordedUpdates { updates, records, consolidated: true }); self.current.clear(); } @@ -466,7 +464,9 @@ pub mod arrangement { impl<'a, U: crate::layout::ColumnarUpdate> timely::container::PushInto<&'a mut RecordedUpdates> for TrieChunker { fn push_into(&mut self, container: &'a mut RecordedUpdates) { - self.ready.push_back(std::mem::take(&mut container.updates)); + let mut updates = std::mem::take(&mut container.updates); + if !container.consolidated { updates = updates.consolidate(); } + if updates.len() > 0 { self.ready.push_back(updates); } } } @@ -497,9 +497,7 @@ pub mod arrangement { use crate::Updates; impl timely::container::SizableContainer for Updates { - fn at_capacity(&self) -> bool { - self.diffs.values.len() >= 64 * 1024 - } + fn at_capacity(&self) -> bool { self.diffs.values.len() >= 64 * 1024 } fn ensure_capacity(&mut self, _stash: &mut Option) { } } @@ -508,7 +506,7 @@ pub mod arrangement { /// TODO: Relax the bound in DD's reduce to remove this requirement. impl InternalMerge for Updates { type TimeOwned = U::Time; - fn len(&self) -> usize { Len::len(&self.keys.values) } + fn len(&self) -> usize { unimplemented!() } fn clear(&mut self) { use columnar::Clear; self.keys.clear(); @@ -516,48 +514,13 @@ pub mod arrangement { self.times.clear(); self.diffs.clear(); } - fn merge_from(&mut self, others: &mut [Self], positions: &mut [usize]) { - let self_iter = self.iter(); - let others_iter = others.iter().zip(positions.iter()).flat_map(|(other, pos)| { - other.iter().skip(*pos) - }); - *self = Self::form_unsorted(self_iter.chain(others_iter)); - for (other, pos) in others.iter().zip(positions.iter_mut()) { - *pos = other.len(); - } - } - fn extract( - &mut self, - upper: AntichainRef, - frontier: &mut Antichain, - keep: &mut Self, - ship: &mut Self, - ) { - let mut time = U::Time::default(); - let mut keep_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); - let mut ship_vec: Vec<(U::Key, U::Val, U::Time, U::Diff)> = Vec::new(); - for (k, v, t, d) in self.iter() { - Columnar::copy_from(&mut time, t); - if upper.less_equal(&time) { - frontier.insert_ref(&time); - keep_vec.push((Columnar::into_owned(k), Columnar::into_owned(v), time.clone(), Columnar::into_owned(d))); - } else { - ship_vec.push((Columnar::into_owned(k), Columnar::into_owned(v), time.clone(), Columnar::into_owned(d))); - } - } - if !keep_vec.is_empty() { - let mut flat = Self::default(); - for (k, v, t, d) in keep.iter() { flat.push((k, v, t, d)); } - for (k, v, t, d) in keep_vec.iter() { flat.push((k, v, t, d)); } - *keep = flat.consolidate(); - } - if !ship_vec.is_empty() { - let mut flat = Self::default(); - for (k, v, t, d) in ship.iter() { flat.push((k, v, t, d)); } - for (k, v, t, d) in ship_vec.iter() { flat.push((k, v, t, d)); } - *ship = flat.consolidate(); - } - } + fn merge_from(&mut self, _others: &mut [Self], _positions: &mut [usize]) { unimplemented!() } + fn extract(&mut self, + _upper: AntichainRef, + _frontier: &mut Antichain, + _keep: &mut Self, + _ship: &mut Self, + ) { unimplemented!() } } } @@ -640,15 +603,7 @@ pub mod arrangement { output: &mut Vec>, _stash: &mut Vec>, ) { - let iter1 = list1.iter().flat_map(|chunk| chunk.iter()); - let iter2 = list2.iter().flat_map(|chunk| chunk.iter()); - - let merged = Merging { - iter1: iter1.peekable(), - iter2: iter2.peekable(), - }; - - form_chunks::(merged, output); + Self::merge_batches(list1, list2, output, _stash); } fn extract( @@ -688,6 +643,507 @@ pub mod arrangement { } } + impl TrieMerger + where + U::Time: Ord + PartialOrder + Clone + 'static, + { + /// Iterator-based merge: flatten, merge, consolidate, form. + /// Correct but slow — used as fallback. + #[allow(dead_code)] + fn merge_iterator( + list1: &[Updates], + list2: &[Updates], + output: &mut Vec>, + ) { + let iter1 = list1.iter().flat_map(|chunk| chunk.iter()); + let iter2 = list2.iter().flat_map(|chunk| chunk.iter()); + + let merged = Merging { + iter1: iter1.peekable(), + iter2: iter2.peekable(), + }; + + form_chunks::(merged, output); + } + + /// A merge implementation that operates batch-at-a-time. + #[inline(never)] + fn merge_batches( + list1: Vec>, + list2: Vec>, + output: &mut Vec>, + stash: &mut Vec>, + ) { + + // The design for efficient "batch" merginging of chains of links is: + // 0. We choose a target link size, K, and will keep the average link size at least K and the max size at 2k. + // K should be large enough to amortize some set-up, but not so large that one or two extra break the bank. + // 1. We will repeatedly consider pairs of links, and fully merge one with a prefix of the other. + // The last elements of each link will tell us which of the two suffixes must be held back. + // 2. We then have a chain of as many links as we started with, with potential defects to correct: + // a. A link may contain some number of zeros: we can remove them if we are eager, based on size. + // b. A link may contain more than 2K updates; we can split it. + // c. Two adjacent links may contain fewer than 2K updates; we can meld (careful append) them. + // 3. After a pass of the above, we should have restored the invariant. + // We can try and me smarter and fuse some of the above work rather than explicitly stage results. + // + // The challenging moment is the merge that can start with a suffix of one link, involving a prefix of one link. + // These could be the same link, different links, and generally there is the potential for complexity here. + + let mut builder = ChainBuilder::default(); + + let mut queue1: std::collections::VecDeque<_> = list1.into(); + let mut queue2: std::collections::VecDeque<_> = list2.into(); + + // The first unconsumed update in each block, via (k_idx, v_idx, t_idx), or None if exhausted. + // These are (0,0,0) for a new block, and should become None once there are no remaining updates. + let mut cursor1 = queue1.pop_front().map(|b| ((0,0,0), b)); + let mut cursor2 = queue2.pop_front().map(|b| ((0,0,0), b)); + + // For each pair of batches + while cursor1.is_some() && cursor2.is_some() { + Self::merge_batch(&mut cursor1, &mut cursor2, &mut builder, stash); + if cursor1.is_none() { cursor1 = queue1.pop_front().map(|b| ((0,0,0), b)); } + if cursor2.is_none() { cursor2 = queue2.pop_front().map(|b| ((0,0,0), b)); } + } + + // TODO: create batch for the non-empty cursor. + if let Some(((k,v,t),batch)) = cursor1 { + let mut out_batch = stash.pop().unwrap_or_default(); + let empty: Updates = Default::default(); + write_from_surveys( + &batch, + &empty, + &[Report::This(0, 1)], + &[Report::This(k, batch.keys.values.len())], + &[Report::This(v, batch.vals.values.len())], + &[Report::This(t, batch.times.values.len())], + &mut out_batch, + ); + builder.push(out_batch); + } + if let Some(((k,v,t),batch)) = cursor2 { + let mut out_batch = stash.pop().unwrap_or_default(); + let empty: Updates = Default::default(); + write_from_surveys( + &empty, + &batch, + &[Report::That(0, 1)], + &[Report::That(k, batch.keys.values.len())], + &[Report::That(v, batch.vals.values.len())], + &[Report::That(t, batch.times.values.len())], + &mut out_batch, + ); + builder.push(out_batch); + } + + builder.extend(queue1); + builder.extend(queue2); + *output = builder.done(); + // TODO: Tidy output to satisfy structural invariants. + } + + /// Merge two batches, one completely and another through the corresponding prefix. + /// + /// Each invocation determines the maximum amount of both batches we can merge, determined + /// by comparing the elements at the tails of each batch, and locating the lesser in other. + /// We will merge the whole of the batch containing the lesser, and the prefix up through + /// the lesser element in the other batch, setting the cursor to the first element strictly + /// greater than that lesser element. + /// + /// The algorithm uses a list of `Report` findings to map the interleavings of the layers. + /// Each indicates either a range exclusive to one of the inputs, or a one element common + /// to the layers from both inputs, which must be further explored. This map would normally + /// allow the full merge to happen, but we need to carefully start at each cursor, and end + /// just before the first element greater than the lesser bound. + /// + /// The consumed prefix and disjoint suffix should be single report entries, and it seems + /// fine to first produce all reports and then reflect on the cursors, rather than use the + /// cursors as part of the mapping. + #[inline(never)] + fn merge_batch( + batch1: &mut Option<((usize, usize, usize), Updates)>, + batch2: &mut Option<((usize, usize, usize), Updates)>, + builder: &mut ChainBuilder, + stash: &mut Vec>, + ) { + let ((k0_idx, v0_idx, t0_idx), updates0) = batch1.take().unwrap(); + let ((k1_idx, v1_idx, t1_idx), updates1) = batch2.take().unwrap(); + + use columnar::Borrow; + let keys0 = updates0.keys.borrow(); + let keys1 = updates1.keys.borrow(); + let vals0 = updates0.vals.borrow(); + let vals1 = updates1.vals.borrow(); + let times0 = updates0.times.borrow(); + let times1 = updates1.times.borrow(); + + // Survey the interleaving of the two inputs. + let mut key_survey = survey::>(keys0, keys1, &[Report::Both(0,0)]); + let mut val_survey = survey::>(vals0, vals1, &key_survey); + let mut time_survey = survey::>(times0, times1, &val_survey); + + // We now know enough to start writing into an output batch. + // We should update the input surveys to reflect the subset + // of data that we want. + // + // At most one cursor should be non-zero (assert!). + // A non-zero cursor must correspond to the first entry of the surveys, + // as there is at least one consumed update that precedes the other batch. + // We need to nudge that report forward to align with the cursor, potentially + // squeezing the report to nothing (to the upper bound). + + // We start by updating the surveys to reflect the cursors. + // If either cursor is set, then its batch has an element strictly less than the other batch. + // We therefore expect to find a prefix of This/That at the start of the survey. + if (k0_idx, v0_idx, t0_idx) != (0,0,0) { + let mut done = false; while !done { if let Report::This(l,u) = &mut key_survey[0] { if *u <= k0_idx { key_survey.remove(0); } else { *l = k0_idx; done = true; } } else { done = true; } } + let mut done = false; while !done { if let Report::This(l,u) = &mut val_survey[0] { if *u <= v0_idx { val_survey.remove(0); } else { *l = v0_idx; done = true; } } else { done = true; } } + let mut done = false; while !done { if let Report::This(l,u) = &mut time_survey[0] { if *u <= t0_idx { time_survey.remove(0); } else { *l = t0_idx; done = true; } } else { done = true; } } + } + + if (k1_idx, v1_idx, t1_idx) != (0,0,0) { + let mut done = false; while !done { if let Report::That(l,u) = &mut key_survey[0] { if *u <= k1_idx { key_survey.remove(0); } else { *l = k1_idx; done = true; } } else { done = true; } } + let mut done = false; while !done { if let Report::That(l,u) = &mut val_survey[0] { if *u <= v1_idx { val_survey.remove(0); } else { *l = v1_idx; done = true; } } else { done = true; } } + let mut done = false; while !done { if let Report::That(l,u) = &mut time_survey[0] { if *u <= t1_idx { time_survey.remove(0); } else { *l = t1_idx; done = true; } } else { done = true; } } + } + + // We want to trim the tails of the surveys to only cover ranges present in both inputs. + // We can determine which was "longer" by looking at the last entry of the bottom layer, + // which tells us which input (or both) contained the last element. + // + // From the bottom layer up, we'll identify the index of the last item, and then determine + // the index of the list it belongs to. We use that index in the next layer, to locate the + // index of the list it belongs to, on upward. + let next_cursor = match time_survey.last().unwrap() { + Report::This(_,_) => { + // Collect the last value indexes known to strictly exceed an entry in the other batch. + let mut t = times0.values.len(); + while let Some(Report::This(l,_)) = time_survey.last() { t = *l; time_survey.pop(); } + let mut v = vals0.values.len(); + while let Some(Report::This(l,_)) = val_survey.last() { v = *l; val_survey.pop(); } + let mut k = keys0.values.len(); + while let Some(Report::This(l,_)) = key_survey.last() { k = *l; key_survey.pop(); } + // Now we may need to correct by nudging down. + if v == times0.len() || times0.bounds.bounds(v).0 > t { v -= 1; } + if k == vals0.len() || vals0.bounds.bounds(k).0 > v { k -= 1; } + Some(Ok((k,v,t))) + } + Report::Both(_,_) => { None } + Report::That(_,_) => { + // Collect the last value indexes known to strictly exceed an entry in the other batch. + let mut t = times1.values.len(); + while let Some(Report::That(l,_)) = time_survey.last() { t = *l; time_survey.pop(); } + let mut v = vals1.values.len(); + while let Some(Report::That(l,_)) = val_survey.last() { v = *l; val_survey.pop(); } + let mut k = keys1.values.len(); + while let Some(Report::That(l,_)) = key_survey.last() { k = *l; key_survey.pop(); } + // Now we may need to correct by nudging down. + if v == times1.len() || times1.bounds.bounds(v).0 > t { v -= 1; } + if k == vals1.len() || vals1.bounds.bounds(k).0 > v { k -= 1; } + Some(Err((k,v,t))) + } + }; + + // Having updated the surveys, we now copy over the ranges they identify. + let mut out_batch = stash.pop().unwrap_or_default(); + // TODO: We should be able to size `out_batch` pretty accurately from the survey. + write_from_surveys(&updates0, &updates1, &[Report::Both(0,0)], &key_survey, &val_survey, &time_survey, &mut out_batch); + builder.push(out_batch); + + match next_cursor { + Some(Ok(kvt)) => { *batch1 = Some((kvt, updates0)); } + Some(Err(kvt)) => {*batch2 = Some((kvt, updates1)); } + None => { } + } + } + + } + + /// Write merged output from four levels of survey reports. + /// + /// Each layer is written independently: `write_layer` handles keys, vals, + /// and times; `write_diffs` handles diff consolidation. + #[inline(never)] + fn write_from_surveys( + updates0: &Updates, + updates1: &Updates, + root_survey: &[Report], + key_survey: &[Report], + val_survey: &[Report], + time_survey: &[Report], + output: &mut Updates, + ) { + use columnar::Borrow; + + write_layer(updates0.keys.borrow(), updates1.keys.borrow(), root_survey, key_survey, &mut output.keys); + write_layer(updates0.vals.borrow(), updates1.vals.borrow(), key_survey, val_survey, &mut output.vals); + write_layer(updates0.times.borrow(), updates1.times.borrow(), val_survey, time_survey, &mut output.times); + write_diffs::(updates0.diffs.borrow(), updates1.diffs.borrow(), time_survey, &mut output.diffs); + } + + /// From two sequences of interleaved lists, map out the interleaving of their values. + /// + /// The sequence of input reports identify constraints on the sorted order of lists in the two inputs, + /// callout out ranges of each that are exclusively order, and elements that have equal prefixes and + /// therefore "overlap" and should be further investigated through the values of the lists. + /// + /// The output should have the same form but for the next layer: subject to the ordering of `reports`, + /// a similar report for the values of the two lists, appropriate for the next layer. + #[inline(never)] + pub fn survey<'a, C: columnar::Container: Ord>>( + lists0: as columnar::Borrow>::Borrowed<'a>, + lists1: as columnar::Borrow>::Borrowed<'a>, + reports: &[Report], + ) -> Vec { + use columnar::Index; + let mut output = Vec::with_capacity(reports.len()); // may grow larger, but at least this large. + for report in reports.iter() { + match report { + Report::This(lower0, upper0) => { + let (new_lower, _) = lists0.bounds.bounds(*lower0); + let (_, new_upper) = lists0.bounds.bounds(*upper0-1); + output.push(Report::This(new_lower, new_upper)); + } + Report::Both(index0, index1) => { + + // Fetch the bounds from the layers. + let (mut lower0, upper0) = lists0.bounds.bounds(*index0); + let (mut lower1, upper1) = lists1.bounds.bounds(*index1); + + // Scour the intersecting range for matches. + while lower0 < upper0 && lower1 < upper1 { + let val0 = lists0.values.get(lower0); + let val1 = lists1.values.get(lower1); + match val0.cmp(&val1) { + std::cmp::Ordering::Less => { + let start = lower0; + lower0 += 1; + gallop(lists0.values, &mut lower0, upper0, |x| x < val1); + output.push(Report::This(start, lower0)); + }, + std::cmp::Ordering::Equal => { + output.push(Report::Both(lower0, lower1)); + lower0 += 1; + lower1 += 1; + }, + std::cmp::Ordering::Greater => { + let start = lower1; + lower1 += 1; + gallop(lists1.values, &mut lower1, upper1, |x| x < val0); + output.push(Report::That(start, lower1)); + }, + } + } + if lower0 < upper0 { output.push(Report::This(lower0, upper0)); } + if lower1 < upper1 { output.push(Report::That(lower1, upper1)); } + + } + Report::That(lower1, upper1) => { + let (new_lower, _) = lists1.bounds.bounds(*lower1); + let (_, new_upper) = lists1.bounds.bounds(*upper1-1); + output.push(Report::That(new_lower, new_upper)); + } + } + } + + output + } + + /// Write one layer of merged output from a list survey and item survey. + /// + /// The list survey describes which lists to produce (from the layer above). + /// The item survey describes how the items within those lists interleave. + /// Both surveys are consumed completely; a mismatch is a bug. + /// + /// Pruning (from cursor adjustments) can affect the first and last list + /// survey entries: the item survey's ranges may not match the natural + /// bounds of those lists. Middle entries are guaranteed unpruned and can + /// be bulk-copied. + #[inline(never)] + pub fn write_layer<'a, C: columnar::Container: Ord>>( + lists0: as columnar::Borrow>::Borrowed<'a>, + lists1: as columnar::Borrow>::Borrowed<'a>, + list_survey: &[Report], + item_survey: &[Report], + output: &mut crate::updates::Lists, + ) { + use columnar::{Container, Index, Len, Push}; + + let mut item_idx = 0; + + for (pos, list_report) in list_survey.iter().enumerate() { + let is_first = pos == 0; + let is_last = pos == list_survey.len() - 1; + let may_be_pruned = is_first || is_last; + + match list_report { + Report::This(lo, hi) => { + let Report::This(item_lo, item_hi) = item_survey[item_idx] else { unreachable!("Expected This in item survey for This list") }; + item_idx += 1; + if may_be_pruned { + // Item range may not match natural bounds; copy items in bulk + // but compute per-list bounds from natural bounds clamped to + // the item range. + let base = output.values.len(); + output.values.extend_from_self(lists0.values, item_lo..item_hi); + for i in *lo..*hi { + let (_, nat_hi) = lists0.bounds.bounds(i); + output.bounds.push((base + nat_hi.min(item_hi) - item_lo) as u64); + } + } else { + output.extend_from_self(lists0, *lo..*hi); + } + } + Report::That(lo, hi) => { + let Report::That(item_lo, item_hi) = item_survey[item_idx] else { unreachable!("Expected That in item survey for That list") }; + item_idx += 1; + if may_be_pruned { + let base = output.values.len(); + output.values.extend_from_self(lists1.values, item_lo..item_hi); + for i in *lo..*hi { + let (_, nat_hi) = lists1.bounds.bounds(i); + output.bounds.push((base + nat_hi.min(item_hi) - item_lo) as u64); + } + } else { + output.extend_from_self(lists1, *lo..*hi); + } + } + Report::Both(i0, i1) => { + // Merge: consume item survey entries until both sides are covered. + let (mut c0, end0) = lists0.bounds.bounds(*i0); + let (mut c1, end1) = lists1.bounds.bounds(*i1); + while (c0 < end0 || c1 < end1) && item_idx < item_survey.len() { + match item_survey[item_idx] { + Report::This(lo, hi) => { + if lo >= end0 { break; } + output.values.extend_from_self(lists0.values, lo..hi); + c0 = hi; + } + Report::That(lo, hi) => { + if lo >= end1 { break; } + output.values.extend_from_self(lists1.values, lo..hi); + c1 = hi; + } + Report::Both(v0, v1) => { + if v0 >= end0 && v1 >= end1 { break; } + output.values.push(lists0.values.get(v0)); + c0 = v0 + 1; + c1 = v1 + 1; + } + } + item_idx += 1; + } + output.bounds.push(output.values.len() as u64); + } + } + } + } + + /// Write the diff layer from a time survey and two diff inputs. + /// + /// The time survey is the item-level survey for the time layer, which + /// doubles as the list survey for diffs (one diff list per time entry). + /// + /// - `This(lo, hi)`: bulk-copy diff lists from input 0. + /// - `That(lo, hi)`: bulk-copy diff lists from input 1. + /// - `Both(t0, t1)`: consolidate the two singleton diffs. Push `[sum]` + /// if non-zero, or an empty list `[]` if they cancel. + #[inline(never)] + pub fn write_diffs( + diffs0: > as columnar::Borrow>::Borrowed<'_>, + diffs1: > as columnar::Borrow>::Borrowed<'_>, + time_survey: &[Report], + output: &mut crate::updates::Lists>, + ) { + use columnar::{Columnar, Container, Index, Len, Push}; + use differential_dataflow::difference::{Semigroup, IsZero}; + + for report in time_survey.iter() { + match report { + Report::This(lo, hi) => { output.extend_from_self(diffs0, *lo..*hi); } + Report::That(lo, hi) => { output.extend_from_self(diffs1, *lo..*hi); } + Report::Both(t0, t1) => { + // Read singleton diffs via list bounds, consolidate. + let (d0_lo, d0_hi) = diffs0.bounds.bounds(*t0); + let (d1_lo, d1_hi) = diffs1.bounds.bounds(*t1); + assert_eq!(d0_hi - d0_lo, 1, "Expected singleton diff list at t0={t0}"); + assert_eq!(d1_hi - d1_lo, 1, "Expected singleton diff list at t1={t1}"); + let mut diff: U::Diff = Columnar::into_owned(diffs0.values.get(d0_lo)); + diff.plus_equals(&Columnar::into_owned(diffs1.values.get(d1_lo))); + if !diff.is_zero() { output.values.push(&diff); } + output.bounds.push(output.values.len() as u64); + } + } + } + } + + /// Increments `index` until just after the last element of `input` to satisfy `cmp`. + /// + /// The method assumes that `cmp` is monotonic, never becoming true once it is false. + /// If an `upper` is supplied, it acts as a constraint on the interval of `input` explored. + #[inline(always)] + pub(crate) fn gallop(input: C, lower: &mut usize, upper: usize, mut cmp: impl FnMut(::Ref) -> bool) { + // if empty input, or already >= element, return + if *lower < upper && cmp(input.get(*lower)) { + let mut step = 1; + while *lower + step < upper && cmp(input.get(*lower + step)) { + *lower += step; + step <<= 1; + } + + step >>= 1; + while step > 0 { + if *lower + step < upper && cmp(input.get(*lower + step)) { + *lower += step; + } + step >>= 1; + } + + *lower += 1; + } + } + + /// A report we would expect to see in a sequence about two layers. + /// + /// A sequence of these reports reveal an ordered traversal of the keys + /// of two layers, with ranges exclusive to one, ranges exclusive to the + /// other, and individual elements (not ranges) common to both. + #[derive(Copy, Clone, Columnar, Debug)] + pub enum Report { + /// Range of indices in this input. + This(usize, usize), + /// Range of indices in that input. + That(usize, usize), + /// Matching indices in both inputs. + Both(usize, usize), + } + + pub struct ChainBuilder { + updates: Vec>, + } + + impl Default for ChainBuilder { fn default() -> Self { Self { updates: Default::default() } } } + + impl ChainBuilder { + fn push(&mut self, mut link: Updates) { + link = link.filter_zero(); + if link.len() > 0 { + if let Some(last) = self.updates.last_mut() { + if last.len() + link.len() < 2 * 64 * 1024 { + let mut build = crate::updates::UpdatesBuilder::new_from(std::mem::take(last)); + build.meld(&link); + *last = build.done(); + } + else { self.updates.push(link); } + + } + else { self.updates.push(link); } + } + } + fn extend(&mut self, iter: impl IntoIterator>) { for link in iter { self.push(link); }} + fn done(self) -> Vec> { self.updates } + } } use builder::ValMirror; @@ -1014,61 +1470,11 @@ pub mod updates { /// Consolidates into canonical trie form: /// single outer key list, all lists sorted and deduplicated, /// diff lists are singletons (or absent if cancelled). - pub fn consolidate(self) -> Self { - Self::form_unsorted(self.iter()) - } + pub fn consolidate(self) -> Self { Self::form_unsorted(self.iter()) } + pub fn filter_zero(self) -> Self { Self::form(self.iter()) } /// The number of leaf-level diff entries (total updates). pub fn len(&self) -> usize { self.diffs.values.len() } - - /// Checks trie invariants and panics with a descriptive message if any fail. - pub fn validate(&self, label: &str) { - let keys_len = Len::len(&self.keys.values); - let vals_bounds_len = Len::len(&self.vals.bounds); - let vals_len = Len::len(&self.vals.values); - let times_bounds_len = Len::len(&self.times.bounds); - let times_len = Len::len(&self.times.values); - let diffs_bounds_len = Len::len(&self.diffs.bounds); - let diffs_len = Len::len(&self.diffs.values); - - // 1. One val-group bound per key. - assert_eq!(vals_bounds_len, keys_len, - "{label}: vals.bounds.len() ({vals_bounds_len}) != keys.values.len() ({keys_len})"); - - // 2. One time-group bound per val. - assert_eq!(times_bounds_len, vals_len, - "{label}: times.bounds.len() ({times_bounds_len}) != vals.values.len() ({vals_len})"); - - // 3. One diff-group bound per time (singleton diffs). - assert_eq!(diffs_bounds_len, times_len, - "{label}: diffs.bounds.len() ({diffs_bounds_len}) != times.values.len() ({times_len})"); - - // 4. Cumulative bounds are monotonically non-decreasing at each level. - // 5. Final bound at each level matches the child values count. - fn check_monotone_and_final + Len>( - bounds: &B, - child_len: usize, - level: &str, - label: &str, - ) { - let n = Len::len(bounds); - if n == 0 { return; } - let mut prev = bounds.index_as(0); - for i in 1..n { - let curr = bounds.index_as(i); - assert!(curr >= prev, - "{label}: {level}.bounds not monotonic at index {i}: {prev} > {curr}"); - prev = curr; - } - let last = bounds.index_as(n - 1) as usize; - assert_eq!(last, child_len, - "{label}: last {level}.bounds ({last}) != child values len ({child_len})"); - } - - check_monotone_and_final(&self.vals.bounds.borrow(), vals_len, "vals", label); - check_monotone_and_final(&self.times.bounds.borrow(), times_len, "times", label); - check_monotone_and_final(&self.diffs.bounds.borrow(), diffs_len, "diffs", label); - } } /// Push a single flat update as a stride-1 entry. @@ -1459,14 +1865,13 @@ where .inner .unary::, _, _, _>(Pipeline, "JoinFunction", move |_, _| { move |input, output| { + let mut t1o = U::Time::default(); + let mut d1o = U::Diff::default(); input.for_each(|time, data| { - if std::env::var("DDIR_VALIDATE").is_ok() { - data.updates.validate("JoinFunction input"); - } let mut session = output.session_with_builder(&time); for (k1, v1, t1, d1) in data.updates.iter() { - let t1o: U::Time = Columnar::into_owned(t1); - let d1o: U::Diff = Columnar::into_owned(d1); + Columnar::copy_from(&mut t1o, t1); + Columnar::copy_from(&mut d1o, d1); for (k2, v2, t2, d2) in logic(k1, v1, t1, d1) { let t3 = t2.join(&t1o); let d3 = d2.multiply(&d1o); @@ -1528,6 +1933,7 @@ where builder.build(move |_capability| { let mut col_builder = ValColBuilder::<(K, V, DynTime, R)>::default(); + let mut time = DynTime::default(); move |_frontier| { let mut output = output.activate(); op_input.for_each(|cap, data| { @@ -1544,7 +1950,7 @@ where // that accepts pre-sorted, potentially-collapsing timestamps // could avoid the re-sort inside the builder. for (k, v, t, d) in data.updates.iter() { - let mut time: DynTime = Columnar::into_owned(t); + Columnar::copy_from(&mut time, t); let mut inner_vec = std::mem::take(&mut time.inner).into_inner(); inner_vec.truncate(level - 1); time.inner = PointStamp::new(inner_vec); From ee080179b9df81de6bac81202f729ccb86192976 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:12:29 -0400 Subject: [PATCH 04/27] Remove commented code --- differential-dataflow/src/operators/reduce.rs | 35 ------------------- 1 file changed, 35 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 77834ab42..6433a3d33 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -479,18 +479,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 @@ -643,24 +631,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. @@ -754,12 +724,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() { From 1535c4b0e3499d165b3de12b996e6e6e44f948c5 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:14:57 -0400 Subject: [PATCH 05/27] Removed one-off trait --- differential-dataflow/src/operators/reduce.rs | 37 ++----------------- 1 file changed, 4 insertions(+), 33 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 6433a3d33..7e03139ff 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -347,35 +347,6 @@ fn sort_dedup(list: &mut Vec) { list.dedup(); } -trait PerKeyCompute<'a, C1, C2, C3, V> -where - C1: Cursor, - C2: for<'b> Cursor = C1::Key<'a>, ValOwn = V, Time = C1::Time>, - C3: Cursor = C1::Key<'a>, Val<'a> = C1::Val<'a>, Time = C1::Time, Diff = C1::Diff>, - V: Clone + Ord, -{ - fn new() -> Self; - fn compute( - &mut self, - key: C1::Key<'a>, - 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)>)], - 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)>, - ); -} - - /// Implementation based on replaying historical and new updates together. mod history_replay { @@ -386,7 +357,7 @@ mod history_replay { use crate::trace::Cursor; use crate::operators::ValueHistory; - use super::{PerKeyCompute, sort_dedup}; + use super::sort_dedup; /// 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. @@ -410,14 +381,14 @@ mod history_replay { 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, V> HistoryReplayer<'a, C1, C2, C3, V> where C1: Cursor, C2: for<'b> Cursor = C1::Key<'a>, ValOwn = V, Time = C1::Time>, C3: Cursor = C1::Key<'a>, Val<'a> = C1::Val<'a>, Time = C1::Time, Diff = C1::Diff>, V: Clone + Ord, { - fn new() -> Self { + pub fn new() -> Self { HistoryReplayer { input_history: ValueHistory::new(), output_history: ValueHistory::new(), @@ -433,7 +404,7 @@ mod history_replay { } } #[inline(never)] - fn compute( + pub fn compute( &mut self, key: C1::Key<'a>, (source_cursor, source_storage): (&mut C1, &'a C1::Storage), From e6af4581d4e838167880867a3f3298478b1444df Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:16:04 -0400 Subject: [PATCH 06/27] Remove unread counters --- differential-dataflow/src/operators/reduce.rs | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 7e03139ff..fdf7bbec3 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -221,7 +221,7 @@ where sort_dedup(&mut interesting_times); // do the per-key computation. - let _counters = thinker.compute( + thinker.compute( key, (&mut source_cursor, source_storage), (&mut output_cursor, output_storage), @@ -414,7 +414,7 @@ mod history_replay { logic: &mut L, upper_limit: &Antichain, outputs: &mut [(C2::Time, Vec<(V, C2::Time, C2::Diff)>)], - new_interesting: &mut Vec) -> (usize, usize) + new_interesting: &mut Vec) where L: FnMut( C1::Key<'a>, @@ -487,9 +487,6 @@ mod history_replay { let mut times_slice = ×[..]; let mut meets_slice = &self.meets[..]; - let mut compute_counter = 0; - let mut output_counter = 0; - // We have candidate times from `batch` and `times`, as well as times identified by either // `input` or `output`. Finally, we may have synthetic times produced as the join of times // we consider in the course of evaluation. As long as any of these times exist, we need to @@ -553,8 +550,6 @@ mod history_replay { // output produced. This sounds like a good test to have for debug builds! if interesting { - compute_counter += 1; - // Assemble the input collection at `next_time`. (`self.input_buffer` cleared just after use). debug_assert!(self.input_buffer.is_empty()); meet.as_ref().map(|meet| input_replay.advance_buffer_by(meet)); @@ -613,8 +608,6 @@ mod history_replay { // times. if !self.update_buffer.is_empty() { - output_counter += 1; - // We *should* be able to find a capability for `next_time`. Any thing else would // indicate a logical error somewhere along the way; either we release a capability // we should have kept, or we have computed the output incorrectly (or both!) @@ -709,8 +702,6 @@ mod history_replay { // Normalize the representation of `new_interesting`, deduplicating and ordering. sort_dedup(new_interesting); - - (compute_counter, output_counter) } } From 8f9e929d9bb68cd5b9fda98344830428ae51c492 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:19:16 -0400 Subject: [PATCH 07/27] Convert silent errors to panics --- differential-dataflow/src/operators/reduce.rs | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index fdf7bbec3..2b4fa29c2 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -83,8 +83,6 @@ where let mut output_upper = Antichain::from_elem(::minimum()); let mut output_lower = Antichain::from_elem(::minimum()); - let id = scope.index(); - move |(input, _frontier), output| { // The `reduce` operator receives fully formed batches, which each serve as an indication @@ -301,15 +299,8 @@ where // Update `capabilities` to reflect interesting pairs described by `frontier`. let mut new_capabilities = Vec::new(); for time in frontier.borrow().iter() { - if let Some(cap) = capabilities.iter().find(|c| c.time().less_equal(time)) { - new_capabilities.push(cap.delayed(time)); - } - else { - println!("{}:\tfailed to find capability less than new frontier time:", id); - println!("{}:\t time: {:?}", id, time); - println!("{}:\t caps: {:?}", id, capabilities); - println!("{}:\t uppr: {:?}", id, upper_limit); - } + let cap = capabilities.iter().find(|c| c.time().less_equal(time)).expect("failed to find capability"); + new_capabilities.push(cap.delayed(time)); } capabilities = new_capabilities; From 51aef09bcd12bda0c94526566c6af790d24601ce Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:27:13 -0400 Subject: [PATCH 08/27] Simplify logic --- differential-dataflow/src/operators/reduce.rs | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 2b4fa29c2..50b4d55ee 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -447,26 +447,18 @@ mod history_replay { // and guaranteed to accumulate identically for times greater or equal to `meet`. // 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); - time.join_assign(meet); - time - }) - } - else { - self.input_history.replay_key(source_cursor, source_storage, key, |time| C1::owned_time(time)) - }; - let mut output_replay = if let Some(meet) = meet.as_ref() { - self.output_history.replay_key(output_cursor, output_storage, key, |time| { - let mut time = C2::owned_time(time); - time.join_assign(meet); - time - }) - } - else { - self.output_history.replay_key(output_cursor, output_storage, key, |time| C2::owned_time(time)) - }; + let mut input_replay = + self.input_history.replay_key(source_cursor, source_storage, key, |time| { + let mut time = C1::owned_time(time); + if let Some(meet) = meet.as_ref() { time.join_assign(meet); } + time + }); + let mut output_replay = + self.output_history.replay_key(output_cursor, output_storage, key, |time| { + let mut time = C2::owned_time(time); + if let Some(meet) = meet.as_ref() { time.join_assign(meet); } + time + }); self.synth_times.clear(); self.times_current.clear(); From 3901fdb9b9928c9350b60e7e959c7928db7061c1 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:30:22 -0400 Subject: [PATCH 09/27] More commented code removed --- differential-dataflow/src/operators/reduce.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 50b4d55ee..596c63e53 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -60,11 +60,8 @@ where let (mut output_reader, mut output_writer) = TraceAgent::new(empty, operator_info, logger); - // let mut output_trace = TraceRc::make_from(agent).0; *result_trace = Some(output_reader.clone()); - // let mut thinker1 = history_replay_prior::HistoryReplayer::::new(); - // let mut thinker = history_replay::HistoryReplayer::::new(); let mut new_interesting_times = Vec::::new(); // Our implementation maintains a list of outstanding `(key, time)` synthetic interesting times, @@ -666,7 +663,7 @@ mod history_replay { // Update `meet` to track the meet of each source of times. - meet = None;//T::maximum(); + meet = None; update_meet(&mut meet, batch_replay.meet()); update_meet(&mut meet, input_replay.meet()); update_meet(&mut meet, output_replay.meet()); From b157359b286bec095611cfc5ba90033a58940cf3 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:30:56 -0400 Subject: [PATCH 10/27] Extract unconditional behavior --- differential-dataflow/src/operators/reduce.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 596c63e53..f9551a3a0 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -300,14 +300,11 @@ where new_capabilities.push(cap.delayed(time)); } capabilities = new_capabilities; - - // ensure that observed progress is reflected in the output. - output_writer.seal(upper_limit.clone()); - } - else { - output_writer.seal(upper_limit.clone()); } + // ensure that observed progress is reflected in the output. + output_writer.seal(upper_limit.clone()); + // We only anticipate future times in advance of `upper_limit`. source_trace.set_logical_compaction(upper_limit.borrow()); output_reader.set_logical_compaction(upper_limit.borrow()); From fadcd7b89b45040533ab007bc9a4a31d1817f715 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:35:13 -0400 Subject: [PATCH 11/27] Idiomatic Rust --- differential-dataflow/src/operators/reduce.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index f9551a3a0..23ad76094 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -529,7 +529,7 @@ mod history_replay { // Assemble the input collection at `next_time`. (`self.input_buffer` cleared just after use). debug_assert!(self.input_buffer.is_empty()); - meet.as_ref().map(|meet| input_replay.advance_buffer_by(meet)); + if let Some(meet) = meet.as_ref() { input_replay.advance_buffer_by(meet) }; for &((value, ref time), ref diff) in input_replay.buffer().iter() { if time.less_equal(&next_time) { self.input_buffer.push((value, diff.clone())); @@ -548,7 +548,7 @@ mod history_replay { } crate::consolidation::consolidate(&mut self.input_buffer); - meet.as_ref().map(|meet| output_replay.advance_buffer_by(meet)); + if let Some(meet) = meet.as_ref() { 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())); From 2ec3293b148b4556637a9ab4e9c4ae4cd5f6bb62 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 04:53:49 -0400 Subject: [PATCH 12/27] Less time cloning --- differential-dataflow/src/operators/reduce.rs | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 23ad76094..3715d2843 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -473,7 +473,7 @@ mod history_replay { input_replay.time(), output_replay.time(), self.synth_times.last(), - ].iter().cloned().flatten().min().cloned() { + ].into_iter().flatten().min().cloned() { // Advance input and output history replayers. This marks applicable updates as active. input_replay.step_while_time_is(&next_time); @@ -511,7 +511,7 @@ mod history_replay { // and become the time itself. They may not equal the current time because whatever frontier we // are tracking may not have advanced far enough. // TODO: `batch_history` may or may not be super compact at this point, and so this check might - // yield false positives if not sufficiently compact. Maybe we should into this and see. + // yield false positives if not sufficiently compact. Maybe we should look into this and see. interesting = interesting || batch_replay.buffer().iter().any(|&((_, ref t),_)| t.less_equal(&next_time)); interesting = interesting || self.times_current.iter().any(|t| t.less_equal(&next_time)); @@ -658,7 +658,6 @@ mod history_replay { debug_assert!(outputs.iter().any(|(t,_)| t.less_equal(&next_time))) } - // Update `meet` to track the meet of each source of times. meet = None; update_meet(&mut meet, batch_replay.meet()); @@ -685,12 +684,8 @@ mod history_replay { /// Updates an optional meet by an optional time. fn update_meet(meet: &mut Option, other: Option<&T>) { if let Some(time) = other { - if let Some(meet) = meet.as_mut() { - *meet = meet.meet(time); - } - if meet.is_none() { - *meet = Some(time.clone()); - } + if let Some(meet) = meet.as_mut() { meet.meet_assign(time); } + else { *meet = Some(time.clone()); } } } } From 81533872e2d317ac8a89382c485d344106779085 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 05:09:47 -0400 Subject: [PATCH 13/27] Idiomatic capability use --- differential-dataflow/src/operators/reduce.rs | 24 ++++--------------- 1 file changed, 5 insertions(+), 19 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 3715d2843..4e3695e8f 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -8,13 +8,11 @@ use timely::container::PushInto; use crate::Data; -use timely::order::PartialOrder; use timely::progress::frontier::Antichain; use timely::progress::Timestamp; use timely::dataflow::*; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; -use timely::dataflow::operators::Capability; use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; @@ -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 capabilities = Vec::>::new(); + let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); // buffers and logic for computing per-key interesting times "efficiently". let mut interesting_times = Vec::::new(); @@ -119,10 +117,7 @@ where } // Ensure that `capabilities` covers the capability of the batch. - capabilities.retain(|cap| !capability.time().less_than(cap.time())); - if !capabilities.iter().any(|cap| cap.time().less_equal(capability.time())) { - capabilities.push(capability.retain(0)); - } + capabilities.insert(capability.retain(0)); }); // Pull in any subsequent empty batches we believe to exist. @@ -287,19 +282,10 @@ where // uses exactly `upper_limit` to determine the upper bound. Good to check though. assert!(output_upper.borrow() == upper_limit.borrow()); - // Determine the frontier of our interesting times. + // Update `capabilities` to reflect interesting times. let mut frontier = Antichain::::new(); - for (_, time) in &interesting { - frontier.insert_ref(time); - } - - // Update `capabilities` to reflect interesting pairs described by `frontier`. - let mut new_capabilities = Vec::new(); - for time in frontier.borrow().iter() { - let cap = capabilities.iter().find(|c| c.time().less_equal(time)).expect("failed to find capability"); - new_capabilities.push(cap.delayed(time)); - } - capabilities = new_capabilities; + for (_, time) in &interesting { frontier.insert_ref(time); } + capabilities.downgrade(frontier); } // ensure that observed progress is reflected in the output. From d0215402dfbf9af407c9ba01c46d934e5aa39362 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 05:19:13 -0400 Subject: [PATCH 14/27] Ref keyword to borrow --- differential-dataflow/src/operators/reduce.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 4e3695e8f..44f3727a1 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -168,12 +168,9 @@ where let mut buffer = Bu::Input::default(); // cursors for navigating input and output traces. - let (mut source_cursor, source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); - let source_storage = &source_storage; - let (mut output_cursor, output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); - let output_storage = &output_storage; - let (mut batch_cursor, batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); - let batch_storage = &batch_storage; + let (mut source_cursor, ref source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); + let (mut output_cursor, ref output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); + let (mut batch_cursor, ref batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); let mut thinker = history_replay::HistoryReplayer::new(); From 7a0283486c5e4d03eea6952127e93cefa2c9dfaa Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 05:53:51 -0400 Subject: [PATCH 15/27] Further tightening --- differential-dataflow/src/operators/reduce.rs | 118 +++++++----------- 1 file changed, 43 insertions(+), 75 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 44f3727a1..650feeb0d 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -469,11 +469,7 @@ mod history_replay { // Advance batch history, and capture whether an update exists at `next_time`. let mut interesting = batch_replay.step_while_time_is(&next_time); - if interesting { - if let Some(meet) = meet.as_ref() { - batch_replay.advance_buffer_by(meet); - } - } + if interesting { if let Some(meet) = meet.as_ref() { batch_replay.advance_buffer_by(meet); } } // advance both `synth_times` and `times_slice`, marking this time interesting if in either. while self.synth_times.last() == Some(&next_time) { @@ -513,81 +509,62 @@ mod history_replay { // Assemble the input collection at `next_time`. (`self.input_buffer` cleared just after use). debug_assert!(self.input_buffer.is_empty()); if let Some(meet) = meet.as_ref() { input_replay.advance_buffer_by(meet) }; - for &((value, ref time), ref diff) in input_replay.buffer().iter() { - if time.less_equal(&next_time) { - self.input_buffer.push((value, diff.clone())); - } - else { - self.temporary.push(next_time.join(time)); - } + for ((value, time), diff) in input_replay.buffer().iter() { + if time.less_equal(&next_time) { self.input_buffer.push((*value, diff.clone())); } + else { self.temporary.push(next_time.join(time)); } } - for &((value, ref time), ref diff) in batch_replay.buffer().iter() { - if time.less_equal(&next_time) { - self.input_buffer.push((value, diff.clone())); - } - else { - self.temporary.push(next_time.join(time)); - } + for ((value, time), diff) in batch_replay.buffer().iter() { + if time.less_equal(&next_time) { self.input_buffer.push((*value, diff.clone())); } + else { self.temporary.push(next_time.join(time)); } } crate::consolidation::consolidate(&mut self.input_buffer); + // Assemble the output collection at `next_time`. (`self.output_buffer` cleared just after use). if let Some(meet) = meet.as_ref() { 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())); - } - else { - self.temporary.push(next_time.join(time)); - } + for ((value, time), diff) in output_replay.buffer().iter() { + if time.less_equal(&next_time) { self.output_buffer.push((C2::owned_val(*value), diff.clone())); } + else { self.temporary.push(next_time.join(time)); } } - for &((ref value, ref time), ref diff) in self.output_produced.iter() { - if time.less_equal(&next_time) { - self.output_buffer.push(((*value).to_owned(), diff.clone())); - } - else { - self.temporary.push(next_time.join(time)); - } + for ((value, time), diff) in self.output_produced.iter() { + if time.less_equal(&next_time) { self.output_buffer.push(((*value).to_owned(), diff.clone())); } + else { self.temporary.push(next_time.join(time)); } } crate::consolidation::consolidate(&mut self.output_buffer); - // Apply user logic if non-empty input and see what happens! + // Apply user logic if non-empty input or output and see what happens! if !self.input_buffer.is_empty() || !self.output_buffer.is_empty() { logic(key, &self.input_buffer[..], &mut self.output_buffer, &mut self.update_buffer); self.input_buffer.clear(); self.output_buffer.clear(); - } - - // 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. - crate::consolidation::consolidate(&mut self.update_buffer); - - // Stash produced updates into both capability-indexed buffers and `output_produced`. - // The two locations are important, in that we will compact `output_produced` as we move - // through times, but we cannot compact the output buffers because we need their actual - // times. - if !self.update_buffer.is_empty() { - - // We *should* be able to find a capability for `next_time`. Any thing else would - // indicate a logical error somewhere along the way; either we release a capability - // we should have kept, or we have computed the output incorrectly (or both!) - let idx = outputs.iter().rev().position(|(time, _)| time.less_equal(&next_time)); - let idx = outputs.len() - idx.expect("failed to find index") - 1; - for (val, diff) in self.update_buffer.drain(..) { - self.output_produced.push(((val.clone(), next_time.clone()), diff.clone())); - outputs[idx].1.push((val, next_time.clone(), diff)); - } - // Advance times in `self.output_produced` and consolidate the representation. - // NOTE: We only do this when we add records; it could be that there are situations - // where we want to consolidate even without changes (because an initially - // large collection can now be collapsed). - if let Some(meet) = meet.as_ref() { - for entry in &mut self.output_produced { - (entry.0).1 = (entry.0).1.join(meet); + // 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. + // + // Stash produced updates into both capability-indexed buffers and `output_produced`. + // The two locations are important, in that we will compact `output_produced` as we move + // through times, but we cannot compact the output buffers because we need their actual + // times. + crate::consolidation::consolidate(&mut self.update_buffer); + if !self.update_buffer.is_empty() { + + // We *should* be able to find a capability for `next_time`. Any thing else would + // indicate a logical error somewhere along the way; either we release a capability + // we should have kept, or we have computed the output incorrectly (or both!) + let idx = outputs.iter().rev().position(|(time, _)| time.less_equal(&next_time)); + let idx = outputs.len() - idx.expect("failed to find index") - 1; + for (val, diff) in self.update_buffer.drain(..) { + self.output_produced.push(((val.clone(), next_time.clone()), diff.clone())); + outputs[idx].1.push((val, next_time.clone(), diff)); } + + // Advance times in `self.output_produced` and consolidate the representation. + // NOTE: We only do this when we add records; it could be that there are situations + // where we want to consolidate even without changes (because an initially + // large collection can now be collapsed). + if let Some(meet) = meet.as_ref() { for entry in &mut self.output_produced { (entry.0).1.join_assign(meet); } } + crate::consolidation::consolidate(&mut self.output_produced); } - crate::consolidation::consolidate(&mut self.output_produced); } } @@ -601,17 +578,8 @@ mod history_replay { // Any time, even uninteresting times, must be joined with the current accumulation of // batch times as well as the current accumulation of `times_current`. - for &((_, ref time), _) in batch_replay.buffer().iter() { - if !time.less_equal(&next_time) { - self.temporary.push(time.join(&next_time)); - } - } - for time in self.times_current.iter() { - if !time.less_equal(&next_time) { - self.temporary.push(time.join(&next_time)); - } - } - + self.temporary.extend(batch_replay.buffer().iter().map(|((_,time),_)| time).filter(|time| !time.less_equal(&next_time)).map(|time| time.join(&next_time))); + self.temporary.extend(self.times_current.iter().filter(|time| !time.less_equal(&next_time)).map(|time| time.join(&next_time))); sort_dedup(&mut self.temporary); // Introduce synthetic times, and re-organize if we add any. From 8e75167dba1ca16d2c89c90b5836666d9d875d0c Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 06:18:31 -0400 Subject: [PATCH 16/27] Tidy explanatory text, and reflect frontier --- differential-dataflow/src/operators/reduce.rs | 37 +++++++------------ 1 file changed, 13 insertions(+), 24 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 650feeb0d..9e159e428 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -78,25 +78,15 @@ where let mut output_upper = Antichain::from_elem(::minimum()); let mut output_lower = Antichain::from_elem(::minimum()); - move |(input, _frontier), output| { + move |(input, frontier), output| { - // The `reduce` operator receives fully formed batches, which each serve as an indication - // that the frontier has advanced to the upper bound of their description. + // The operator receives input batches, which it treats as contiguous and will collect and + // then process as one batch. It captures the input frontier from the batches, from the upstream + // trace, and from the input frontier, and retires the work through that interval. // - // Although we could act on each individually, several may have been sent, and it makes - // sense to accumulate them first to coordinate their re-evaluation. We will need to pay - // attention to which times need to be collected under which capability, so that we can - // assemble output batches correctly. We will maintain several builders concurrently, and - // place output updates into the appropriate builder. - // - // It turns out we must use notificators, as we cannot await empty batches from arrange to - // indicate progress, as the arrange may not hold the capability to send such. Instead, we - // must watch for progress here (and the upper bound of received batches) to tell us how - // far we can process work. - // - // We really want to retire all batches we receive, so we want a frontier which reflects - // both information from batches as well as progress information. I think this means that - // we keep times that are greater than or equal to a time in the other frontier, deduplicated. + // Reduce may retain capabilities and need to perform work and produce output at times that + // may not be seen in its input. The standard example is that updates at `(0, 1)` and `(1, 0)` + // may result in outputs at `(1, 1)` as well, even with no input at that time. let mut batch_cursors = Vec::new(); let mut batch_storage = Vec::new(); @@ -105,23 +95,22 @@ where lower_limit.clear(); lower_limit.extend(upper_limit.borrow().iter().cloned()); - // Drain the input stream of batches, validating the contiguity of the batch descriptions and - // capturing a cursor for each of the batches as well as ensuring we hold a capability for the - // times in the batch. + // Drain input batches in order, capturing capabilities and the last upper. input.for_each(|capability, batches| { - + capabilities.insert(capability.retain(0)); for batch in batches.drain(..) { upper_limit.clone_from(batch.upper()); batch_cursors.push(batch.cursor()); batch_storage.push(batch); } - - // Ensure that `capabilities` covers the capability of the batch. - capabilities.insert(capability.retain(0)); }); // Pull in any subsequent empty batches we believe to exist. source_trace.advance_upper(&mut upper_limit); + // Incorporate the input frontier guarantees as well. + let mut joined = Antichain::new(); + crate::lattice::antichain_join_into(&upper_limit.borrow()[..], &frontier.frontier()[..], &mut joined); + upper_limit = joined; // Only if our upper limit has advanced should we do work. if upper_limit != lower_limit { From 7c053ade8bb2afe6183a95860d076e99296fa998 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 06:46:33 -0400 Subject: [PATCH 17/27] Tighten comments, remove mutable borrow --- differential-dataflow/src/operators/reduce.rs | 48 ++++++------------- 1 file changed, 14 insertions(+), 34 deletions(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 9e159e428..f6e5ad60b 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -112,18 +112,15 @@ where crate::lattice::antichain_join_into(&upper_limit.borrow()[..], &frontier.frontier()[..], &mut joined); upper_limit = joined; - // Only if our upper limit has advanced should we do work. + // We plan to retire the interval [lower_limit, upper_limit), which should be non-empty to proceed. if upper_limit != lower_limit { - // If we have no capabilities, then we (i) should not produce any outputs and (ii) could not send - // any produced outputs even if they were (incorrectly) produced. We cannot even send empty batches - // to indicate forward progress, and must hope that downstream operators look at progress frontiers - // as well as batch descriptions. - // - // We can (and should) advance source and output traces if `upper_limit` indicates this is possible. + // If we hold no capabilitys in the interval [lower_limit, upper_limit) then we have no compute needs, + // and could not transmit the outputs even if they were (incorrectly) non-zero. + // We do have maintenance work after this logic, and should not fuse this test with the above test. if capabilities.iter().any(|c| !upper_limit.less_equal(c.time())) { - // `interesting` contains "warnings" about keys and times that may need to be re-considered. + // `interesting` contains "todos" about key and time pairs that should be re-considered. // 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` @@ -139,12 +136,6 @@ where }); // Prepare an output buffer and builder for each capability. - // - // We buffer and build separately, as outputs are produced grouped by time, whereas the - // builder wants to see outputs grouped by value. While the per-key computation could - // do the re-sorting itself, buffering per-key outputs lets us double check the results - // against other implementations for accuracy. - // // 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(); @@ -163,11 +154,7 @@ where let mut thinker = history_replay::HistoryReplayer::new(); - // We now march through the keys we must work on, drawing from `batch_cursors` and `exposed`. - // - // We only keep valid cursors (those with more data) in `batch_cursors`, and so its length - // indicates whether more data remain. We move through `exposed` using (index) `exposed_position`. - // There could perhaps be a less provocative variable name. + // March through the keys we must work on, merging `batch_cursors` and `exposed`. let mut exposed_position = 0; while batch_cursor.key_valid(batch_storage) || exposed_position < exposed_keys.len() { @@ -181,13 +168,9 @@ where (None, None) => unreachable!(), }; - // `interesting_times` contains those times between `lower_issued` and `upper_limit` - // that we need to re-consider. We now populate it, but perhaps this should be left - // to the per-key computation, which may be able to avoid examining the times of some - // values (for example, in the case of min/max/topk). + // Populate `interesting_times` with interesting times not beyond `upper_limit`. + // TODO: This could just be `exposed_time` and `lower .. upper` bounds. interesting_times.clear(); - - // 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))); exposed_position += 1; @@ -202,22 +185,19 @@ where (&mut source_cursor, source_storage), (&mut output_cursor, output_storage), (&mut batch_cursor, batch_storage), - &mut interesting_times, + &interesting_times, &mut logic, &upper_limit, &mut buffers[..], &mut new_interesting_times, ); - if batch_cursor.get_key(batch_storage) == Some(key) { - batch_cursor.step_key(batch_storage); - } + // Advance the cursor if this key, so that the loop's validity check registers the work as done. + if batch_cursor.get_key(batch_storage) == Some(key) { batch_cursor.step_key(batch_storage); } // 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)); - } + debug_assert!(new_interesting_times.iter().all(|t| upper_limit.less_equal(t))); + interesting.extend(new_interesting_times.drain(..).map(|t| (T1::owned_key(key), t))); // Sort each buffer by value and move into the corresponding builder. // TODO: This makes assumptions about at least one of (i) the stability of `sort_by`, @@ -367,7 +347,7 @@ mod history_replay { (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, + times: &Vec, logic: &mut L, upper_limit: &Antichain, outputs: &mut [(C2::Time, Vec<(V, C2::Time, C2::Diff)>)], From 5281266452aea5e6894dafc02c31409342aa6d6f Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 07:00:05 -0400 Subject: [PATCH 18/27] Prefer insert_ref to insert --- differential-dataflow/src/operators/reduce.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index f6e5ad60b..b5b2aac4c 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -227,7 +227,7 @@ where output_upper.clear(); output_upper.extend(upper_limit.borrow().iter().cloned()); for capability in &capabilities[index + 1 ..] { - output_upper.insert(capability.time().clone()); + output_upper.insert_ref(capability.time()); } if output_upper.borrow() != output_lower.borrow() { From 671de13808b4500a86238c254196226aed2e0a88 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 4 Apr 2026 17:15:46 -0400 Subject: [PATCH 19/27] Use containers for interesting (keys, time) (#710) * Use containers for interesting (keys, time) * Remove use of owned keys in reduce.rs * Remove all uses of KeyOwn * Remove KeyOwn * Improve documentation * Walk back overly prescriptive dogs^3 constraints --- .../examples/columnar/main.rs | 12 +- differential-dataflow/src/collection.rs | 28 ++- .../src/operators/arrange/agent.rs | 6 +- .../src/operators/arrange/arrangement.rs | 37 ++-- .../src/operators/arrange/upsert.rs | 24 +-- differential-dataflow/src/operators/reduce.rs | 169 ++++++++++-------- .../src/trace/implementations/mod.rs | 8 +- differential-dataflow/src/trace/mod.rs | 3 - differential-dataflow/tests/import.rs | 12 +- dogsdogsdogs/src/operators/count.rs | 3 +- dogsdogsdogs/src/operators/half_join.rs | 9 +- dogsdogsdogs/src/operators/half_join2.rs | 6 +- dogsdogsdogs/src/operators/lookup_map.rs | 2 +- dogsdogsdogs/src/operators/propose.rs | 4 +- dogsdogsdogs/src/operators/validate.rs | 2 +- interactive/src/plan/mod.rs | 6 +- 16 files changed, 200 insertions(+), 131 deletions(-) diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs index 8404e7aa4..85b573157 100644 --- a/differential-dataflow/examples/columnar/main.rs +++ b/differential-dataflow/examples/columnar/main.rs @@ -168,8 +168,16 @@ mod reachability { let result = combined_arr.reduce_abelian::<_, ValBuilder, ValSpine, - >("Distinct", |_node, _input, output| { - output.push(((), 1)); + _, + >("Distinct", |_node, _input, output| { output.push(((), 1)); }, + |col, key, upds| { + use columnar::{Clear, Push}; + col.keys.clear(); + col.vals.clear(); + col.times.clear(); + col.diffs.clear(); + for (val, time, diff) in upds.drain(..) { col.push((key, &val, &time, &diff)); } + *col = std::mem::take(col).consolidate(); }); // Extract RecordedUpdates from the Arranged's batch stream. diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index c13227f2c..6981cc3da 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -752,7 +752,11 @@ pub mod vec { use crate::trace::implementations::{ValBuilder, ValSpine}; self.arrange_by_key_named(&format!("Arrange: {}", name)) - .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine>(name, logic) + .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine,_>( + name, + logic, + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ) .as_collection(|k,v| (k.clone(), v.clone())) } @@ -782,7 +786,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: for<'a> Trace= &'a K, ValOwn = V, Time=G::Timestamp, Diff: Abelian>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V, T2::Diff)>)+'static, { @@ -801,12 +805,16 @@ 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: for<'a> Trace=&'a K, ValOwn = V, Time=G::Timestamp>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, { self.arrange_by_key_named(&format!("Arrange: {}", name)) - .reduce_core::<_,Bu,_>(name, logic) + .reduce_core::<_,Bu,_,_>( + name, + logic, + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ) } } @@ -871,7 +879,11 @@ pub mod vec { use crate::trace::implementations::{KeyBuilder, KeySpine}; self.arrange_by_self_named(&format!("Arrange: {}", name)) - .reduce_abelian::<_,KeyBuilder,KeySpine>(name, move |k,s,t| t.push(((), thresh(k, &s[0].1)))) + .reduce_abelian::<_,KeyBuilder,KeySpine,_>( + name, + move |k,s,t| t.push(((), thresh(k, &s[0].1))), + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ) .as_collection(|k,_| k.clone()) } @@ -908,7 +920,11 @@ pub mod vec { pub fn count_core + 'static>(self) -> Collection { use crate::trace::implementations::{ValBuilder, ValSpine}; self.arrange_by_self_named("Arrange: Count") - .reduce_abelian::<_,ValBuilder,ValSpine>("Count", |_k,s,t| t.push((s[0].1.clone(), R2::from(1i8)))) + .reduce_abelian::<_,ValBuilder,ValSpine,_>( + "Count", + |_k,s,t| t.push((s[0].1.clone(), R2::from(1i8))), + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ) .as_collection(|k,c| (k.clone(), c.clone())) } } diff --git a/differential-dataflow/src/operators/arrange/agent.rs b/differential-dataflow/src/operators/arrange/agent.rs index e1c2fde5f..715542f55 100644 --- a/differential-dataflow/src/operators/arrange/agent.rs +++ b/differential-dataflow/src/operators/arrange/agent.rs @@ -205,7 +205,11 @@ impl TraceAgent { /// // create a second dataflow /// worker.dataflow(move |scope| { /// trace.import(scope) - /// .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>>("Reduce", |_key, src, dst| dst.push((*src[0].0, 1))) + /// .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>,_>( + /// "Reduce", + /// |_key, src, dst| dst.push((*src[0].0, 1)), + /// |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + /// ) /// .as_collection(|k,v| (k.clone(), v.clone())); /// }); /// diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 69965e65b..cd1bd0894 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -30,7 +30,6 @@ use crate::{Data, VecCollection, AsCollection}; use crate::difference::Semigroup; use crate::lattice::Lattice; use crate::trace::{self, Trace, TraceReader, BatchReader, Batcher, Builder, Cursor}; -use crate::trace::implementations::merge_batcher::container::InternalMerge; use trace::wrappers::enter::{TraceEnter, BatchEnter,}; use trace::wrappers::enter_at::TraceEnter as TraceEnterAt; @@ -75,7 +74,6 @@ where use ::timely::dataflow::scopes::Child; use ::timely::progress::timestamp::Refines; use timely::Container; -use timely::container::PushInto; impl Arranged where @@ -169,12 +167,17 @@ 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 + /// + /// The method takes `K` and `V` as generic arguments, in order to constrain the reference types to support + /// cloning into owned types. If this bound does not work, the `as_collection` method allows arbitrary logic + /// on the reference types. + pub fn as_vecs(self) -> VecCollection where - Tr::KeyOwn: crate::ExchangeData, - Tr::ValOwn: crate::ExchangeData, + K: crate::ExchangeData, + V: crate::ExchangeData, + Tr: for<'a> TraceReader = &'a K, Val<'a> = &'a V>, { - 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 `VecCollection`. @@ -271,43 +274,43 @@ where T1: TraceReader + Clone + 'static, { /// A direct implementation of `ReduceCore::reduce_abelian`. - pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L, push: P) -> Arranged> where - T1: TraceReader, + T1: TraceReader, T2: for<'a> Trace< Key<'a>= T1::Key<'a>, - KeyOwn=T1::KeyOwn, ValOwn: Data, Time=T1::Time, Diff: Abelian, >+'static, - Bu: Builder>, + Bu: Builder, L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, { - self.reduce_core::<_,Bu,T2>(name, move |key, input, output, change| { + self.reduce_core::<_,Bu,T2,_>(name, move |key, input, output, change| { if !input.is_empty() { logic(key, input, change); } change.extend(output.drain(..).map(|(x,mut d)| { d.negate(); (x, d) })); crate::consolidation::consolidate(change); - }) + }, push) } /// A direct implementation of `ReduceCore::reduce_core`. - pub fn reduce_core(self, name: &str, logic: L) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L, push: P) -> Arranged> where - T1: TraceReader, + T1: TraceReader, T2: for<'a> Trace< Key<'a>=T1::Key<'a>, - KeyOwn=T1::KeyOwn, ValOwn: Data, Time=T1::Time, >+'static, - Bu: Builder>, + Bu: Builder, L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, { use crate::operators::reduce::reduce_trace; - reduce_trace::<_,_,Bu,_,_>(self, name, logic) + reduce_trace::<_,_,Bu,_,_,_>(self, name, logic, push) } } diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index e9dbe9cdb..a8322eaa3 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -59,7 +59,7 @@ //! use differential_dataflow::operators::arrange::upsert; //! //! let stream = scope.input_from(&mut input); -//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine>(stream, &"test"); +//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine,String,String>(stream, &"test"); //! //! arranged //! .as_collection(|k,v| (k.clone(), v.clone())) @@ -127,19 +127,21 @@ use super::TraceAgent; /// This method is only implemented for totally ordered times, as we do not yet /// understand what a "sequence" of upserts would mean for partially ordered /// timestamps. -pub fn arrange_from_upsert( - stream: Stream, G::Timestamp)>>, +pub fn arrange_from_upsert( + stream: Stream, G::Timestamp)>>, name: &str, ) -> Arranged> where G: Scope, + K: ExchangeData+Hashable+std::hash::Hash, + V: ExchangeData, Tr: for<'a> Trace< - KeyOwn: ExchangeData+Hashable+std::hash::Hash, - ValOwn: ExchangeData, + Key<'a> = &'a K, + Val<'a> = &'a V, Time: TotalOrder+ExchangeData, Diff=isize, >+'static, - Bu: Builder, Output = Tr::Batch>, + Bu: Builder, Output = Tr::Batch>, { let mut reader: Option> = None; @@ -148,7 +150,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: &(K,Option,G::Timestamp)| (update.0).hashed().into()); let scope = stream.scope(); stream.unary_frontier(exchange, name, move |_capability, info| { @@ -174,7 +176,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| { @@ -237,10 +239,10 @@ where let mut key_con = Tr::KeyContainer::with_capacity(1); for (key, mut list) in to_process { - key_con.clear(); key_con.push_own(&key); + key_con.clear(); key_con.push_ref(&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)); @@ -252,7 +254,7 @@ where 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/reduce.rs b/differential-dataflow/src/operators/reduce.rs index b5b2aac4c..909199295 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -5,7 +5,6 @@ //! to the key and the list of values. //! The function is expected to populate a list of output values. -use timely::container::PushInto; use crate::Data; use timely::progress::frontier::Antichain; @@ -18,21 +17,29 @@ 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::InternalMerge; use crate::trace::TraceReader; -// TODO: Remove the InternalMerge constraint on Bu::Input. It only needs Clear. - /// A key-wise reduction of values in an input trace. /// /// This method exists to provide reduce functionality without opinions about qualifying trace types. -pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L) -> Arranged> +/// +/// The `logic` closure is expected to take a key, accumulated input, and tentative accumulated output, +/// and populate its final argument with whatever it feels to be appopriate updates. The behavior and +/// correctness of the implementation rely on this making sense, and e.g. ideally the updates would if +/// applied to the tentative output bring it in line with some function applied to the input. +/// +/// The `push` closure is expected to clear its first argument, then populate it with the key and drain +/// the value updates, as appropriate for the container. It is critical that it clear the container as +/// the operator has no ability to do this otherwise, and failing to do so represents a leak from one +/// key's computation to another, and will likely introduce non-determinism. +pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L, mut push: P) -> 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>, + T1: TraceReader + Clone + 'static, + T2: for<'a> Trace=T1::Key<'a>, 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, + P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, { let mut result_trace = None; @@ -53,7 +60,6 @@ where empty.set_exert_logic(exert_logic); } - let mut source_trace = trace.trace.clone(); let (mut output_reader, mut output_writer) = TraceAgent::new(empty, operator_info, logger); @@ -63,8 +69,11 @@ where let mut new_interesting_times = Vec::::new(); // 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(); + // sorted by (key, time), as well as capabilities for the lower envelope of the times. + let mut pending_keys = T1::KeyContainer::with_capacity(0); + let mut pending_time = T1::TimeContainer::with_capacity(0); + let mut next_pending_keys = T1::KeyContainer::with_capacity(0); + let mut next_pending_time = T1::TimeContainer::with_capacity(0); let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); // buffers and logic for computing per-key interesting times "efficiently". @@ -115,25 +124,15 @@ where // We plan to retire the interval [lower_limit, upper_limit), which should be non-empty to proceed. if upper_limit != lower_limit { - // If we hold no capabilitys in the interval [lower_limit, upper_limit) then we have no compute needs, + // If we hold no capabilities in the interval [lower_limit, upper_limit) then we have no compute needs, // and could not transmit the outputs even if they were (incorrectly) non-zero. // We do have maintenance work after this logic, and should not fuse this test with the above test. if capabilities.iter().any(|c| !upper_limit.less_equal(c.time())) { - // `interesting` contains "todos" about key and time pairs that should be re-considered. - // 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); - // 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); - false - } - }); + // cursors for navigating input and output traces. + let (mut source_cursor, ref source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); + let (mut output_cursor, ref output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); + let (mut batch_cursor, ref batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); // Prepare an output buffer and builder for each capability. // TODO: It would be better if all updates went into one batch, but timely dataflow prevents @@ -144,22 +143,21 @@ where buffers.push((cap.time().clone(), Vec::new())); builders.push(Bu::new()); } - + // Temporary staging for output building. let mut buffer = Bu::Input::default(); - // cursors for navigating input and output traces. - let (mut source_cursor, ref source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); - let (mut output_cursor, ref output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); - let (mut batch_cursor, ref batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); - + // Reuseable state for performing the computation. let mut thinker = history_replay::HistoryReplayer::new(); + // Merge the received batch cursor with our list of interesting (key, time) moments. + // The interesting moments need to be in the interval to prompt work. + // March through the keys we must work on, merging `batch_cursors` and `exposed`. - let mut exposed_position = 0; - while batch_cursor.key_valid(batch_storage) || exposed_position < exposed_keys.len() { + let mut pending_pos = 0; + while batch_cursor.key_valid(batch_storage) || pending_pos < pending_keys.len() { // Determine the next key we will work on; could be synthetic, could be from a batch. - let key1 = exposed_keys.get(exposed_position); + let key1 = pending_keys.get(pending_pos); let key2 = batch_cursor.get_key(batch_storage); let key = match (key1, key2) { (Some(key1), Some(key2)) => ::std::cmp::min(key1, key2), @@ -169,49 +167,71 @@ where }; // Populate `interesting_times` with interesting times not beyond `upper_limit`. - // TODO: This could just be `exposed_time` and `lower .. upper` bounds. + // TODO: This could just be `pending_time` and indexes within `lower .. upper`. + let prior_pos = pending_pos; interesting_times.clear(); - while exposed_keys.get(exposed_position) == Some(key) { - interesting_times.push(T1::owned_time(exposed_time.index(exposed_position))); - exposed_position += 1; + while pending_keys.get(pending_pos) == Some(key) { + let owned_time = T1::owned_time(pending_time.index(pending_pos)); + if !upper_limit.less_equal(&owned_time) { interesting_times.push(owned_time); } + pending_pos += 1; } // tidy up times, removing redundancy. sort_dedup(&mut interesting_times); - // do the per-key computation. - thinker.compute( - key, - (&mut source_cursor, source_storage), - (&mut output_cursor, output_storage), - (&mut batch_cursor, batch_storage), - &interesting_times, - &mut logic, - &upper_limit, - &mut buffers[..], - &mut new_interesting_times, - ); - - // Advance the cursor if this key, so that the loop's validity check registers the work as done. - if batch_cursor.get_key(batch_storage) == Some(key) { batch_cursor.step_key(batch_storage); } - - // Record future warnings about interesting times (and assert they should be "future"). - debug_assert!(new_interesting_times.iter().all(|t| upper_limit.less_equal(t))); - interesting.extend(new_interesting_times.drain(..).map(|t| (T1::owned_key(key), t))); - - // Sort each buffer by value and move into the corresponding builder. - // TODO: This makes assumptions about at least one of (i) the stability of `sort_by`, - // (ii) that the buffers are time-ordered, and (iii) that the builders accept - // arbitrarily ordered times. - 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)); + // If there are new updates, or pending times, we must investigate! + if batch_cursor.get_key(batch_storage) == Some(key) || !interesting_times.is_empty() { + + // do the per-key computation. + thinker.compute( + key, + (&mut source_cursor, source_storage), + (&mut output_cursor, output_storage), + (&mut batch_cursor, batch_storage), + &interesting_times, + &mut logic, + &upper_limit, + &mut buffers[..], + &mut new_interesting_times, + ); + + // Advance the cursor if this key, so that the loop's validity check registers the work as done. + if batch_cursor.get_key(batch_storage) == Some(key) { batch_cursor.step_key(batch_storage); } + + // Merge novel pending times with any prior pending times we did not process. + // TODO: This could be a merge, not a sort_dedup, because both lists should be sorted. + for pos in prior_pos .. pending_pos { + let owned_time = T1::owned_time(pending_time.index(pos)); + if upper_limit.less_equal(&owned_time) { new_interesting_times.push(owned_time); } + } + sort_dedup(&mut new_interesting_times); + for time in new_interesting_times.drain(..) { + next_pending_keys.push_ref(key); + next_pending_time.push_own(&time); + } + + // Sort each buffer by value and move into the corresponding builder. + // TODO: This makes assumptions about at least one of (i) the stability of `sort_by`, + // (ii) that the buffers are time-ordered, and (iii) that the builders accept + // arbitrarily ordered times. + for index in 0 .. buffers.len() { + buffers[index].1.sort_by(|x,y| x.0.cmp(&y.0)); + push(&mut buffer, key, &mut buffers[index].1); + buffers[index].1.clear(); builders[index].push(&mut buffer); - buffer.clear(); + + } + } + else { + // copy over the pending key and times. + for pos in prior_pos .. pending_pos { + next_pending_keys.push_ref(pending_keys.index(pos)); + next_pending_time.push_ref(pending_time.index(pos)); } } } + // Drop to avoid lifetime issues that would lock `pending_{keys, time}`. + drop(thinker); // We start sealing output batches from the lower limit (previous upper limit). // In principle, we could update `lower_limit` itself, and it should arrive at @@ -243,14 +263,21 @@ where output_lower.extend(output_upper.borrow().iter().cloned()); } } - // This should be true, as the final iteration introduces no capabilities, and // uses exactly `upper_limit` to determine the upper bound. Good to check though. assert!(output_upper.borrow() == upper_limit.borrow()); - // Update `capabilities` to reflect interesting times. + // Refresh pending keys and times, then downgrade capabilities to the frontier of times. + pending_keys.clear(); std::mem::swap(&mut next_pending_keys, &mut pending_keys); + pending_time.clear(); std::mem::swap(&mut next_pending_time, &mut pending_time); + + // Update `capabilities` to reflect pending times. let mut frontier = Antichain::::new(); - for (_, time) in &interesting { frontier.insert_ref(time); } + let mut owned_time = T1::Time::minimum(); + for pos in 0 .. pending_time.len() { + T1::clone_time_onto(pending_time.index(pos), &mut owned_time); + frontier.insert_ref(&owned_time); + } capabilities.downgrade(frontier); } diff --git a/differential-dataflow/src/trace/implementations/mod.rs b/differential-dataflow/src/trace/implementations/mod.rs index d73eb71f4..082af6812 100644 --- a/differential-dataflow/src/trace/implementations/mod.rs +++ b/differential-dataflow/src/trace/implementations/mod.rs @@ -113,8 +113,6 @@ pub trait WithLayout { /// 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. @@ -131,7 +129,7 @@ pub trait LayoutExt : WithLayout: Copy + Ord; /// Container for update keys. - type KeyContainer: for<'a> BatchContainer = Self::Key<'a>, Owned = Self::KeyOwn>; + type KeyContainer: for<'a> BatchContainer = Self::Key<'a>>; /// Container for update vals. type ValContainer: for<'a> BatchContainer = Self::Val<'a>, Owned = Self::ValOwn>; /// Container for times. @@ -139,8 +137,6 @@ pub trait LayoutExt : WithLayout 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. @@ -153,7 +149,6 @@ pub trait LayoutExt : WithLayout 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>; @@ -167,7 +162,6 @@ impl LayoutExt for L { 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) } diff --git a/differential-dataflow/src/trace/mod.rs b/differential-dataflow/src/trace/mod.rs index 0197e09c0..582a6da62 100644 --- a/differential-dataflow/src/trace/mod.rs +++ b/differential-dataflow/src/trace/mod.rs @@ -54,7 +54,6 @@ pub trait TraceReader : LayoutExt { WithLayout + for<'a> LayoutExt< Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, Val<'a> = Self::Val<'a>, ValOwn = Self::ValOwn, Time = Self::Time, @@ -77,7 +76,6 @@ pub trait TraceReader : LayoutExt { WithLayout + for<'a> LayoutExt< Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, Val<'a> = Self::Val<'a>, ValOwn = Self::ValOwn, Time = Self::Time, @@ -253,7 +251,6 @@ pub trait BatchReader : LayoutExt + Sized { WithLayout + for<'a> LayoutExt< Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, Val<'a> = Self::Val<'a>, ValOwn = Self::ValOwn, Time = Self::Time, diff --git a/differential-dataflow/tests/import.rs b/differential-dataflow/tests/import.rs index 747e45b72..1bb2c9cc1 100644 --- a/differential-dataflow/tests/import.rs +++ b/differential-dataflow/tests/import.rs @@ -60,7 +60,11 @@ fn test_import_vanilla() { ::std::mem::drop(trace); let captured = imported - .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine>("Reduce", |_k, s, t| t.push((s.iter().map(|&(_, w)| w).sum(), 1i64))) + .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine,_>( + "Reduce", + |_k, s, t| t.push((s.iter().map(|&(_, w)| w).sum(), 1i64)), + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ) .as_collection(|k,v| (k.clone(), v.clone())) .inner .exchange(|_| 0) @@ -135,7 +139,11 @@ fn test_import_completed_dataflow() { ::std::mem::drop(trace); let stream = imported - .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine>("Reduce", |_k, s, t| t.push((s.iter().map(|&(_, w)| w).sum(), 1i64))) + .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine,_>( + "Reduce", + |_k, s, t| t.push((s.iter().map(|&(_, w)| w).sum(), 1i64)), + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); + }) .as_collection(|k,v| (k.clone(), v.clone())) .inner .exchange(|_| 0); diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index 0cd363a76..42367a2e1 100644 --- a/dogsdogsdogs/src/operators/count.rs +++ b/dogsdogsdogs/src/operators/count.rs @@ -19,7 +19,8 @@ pub fn count( ) -> VecCollection where G: Scope, - Tr: TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, + Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, for<'a> Tr::Diff : Semigroup>, K: Hashable + Ord + Default + 'static, R: Monoid+Multiply+ExchangeData, diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index ddb2a5979..2a18a9aa1 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -85,7 +85,8 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, + Tr::KeyContainer: BatchContainer, R: Mul, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, @@ -151,7 +152,8 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: for<'a> TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, + Tr::KeyContainer: BatchContainer, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, @@ -312,7 +314,8 @@ fn process_proposals( ) -> bool where G: Scope, - Tr: for<'a> TraceReader, + Tr: TraceReader, + Tr::KeyContainer: BatchContainer, CF: Fn(Tr::TimeGat<'_>, &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, diff --git a/dogsdogsdogs/src/operators/half_join2.rs b/dogsdogsdogs/src/operators/half_join2.rs index 650dc433d..08029f96e 100644 --- a/dogsdogsdogs/src/operators/half_join2.rs +++ b/dogsdogsdogs/src/operators/half_join2.rs @@ -73,7 +73,8 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, + Tr::KeyContainer: BatchContainer, R: Mul, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, @@ -130,7 +131,8 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: for<'a> TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, + Tr::KeyContainer: BatchContainer, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index 7908ce858..65108ed59 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -29,10 +29,10 @@ pub fn lookup_map( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = K, Time: std::hash::Hash, Diff : Semigroup>+Monoid+ExchangeData, >+Clone+'static, + Tr::KeyContainer: BatchContainer, K: Hashable + Ord + 'static, F: FnMut(&D, &mut K)+Clone+'static, D: ExchangeData, diff --git a/dogsdogsdogs/src/operators/propose.rs b/dogsdogsdogs/src/operators/propose.rs index 9f2e69f78..6cdcb130d 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -21,11 +21,11 @@ pub fn propose( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = K, ValOwn = V, Time: std::hash::Hash, Diff: Monoid+Multiply+ExchangeData+Semigroup>, >+Clone+'static, + Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, K: Hashable + Default + Ord + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, @@ -55,11 +55,11 @@ pub fn propose_distinct( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = K, ValOwn = V, Time: std::hash::Hash, Diff : Semigroup>+Monoid+Multiply+ExchangeData, >+Clone+'static, + Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, K: Hashable + Default + Ord + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index f610d55be..0ab7297bb 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -20,10 +20,10 @@ pub fn validate( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = (K, V), Time: std::hash::Hash, Diff : Semigroup>+Monoid+Multiply+ExchangeData, >+Clone+'static, + Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, K: Ord+Hash+Clone+Default + 'static, V: ExchangeData+Hash+Default, F: Fn(&P)->K+Clone+'static, diff --git a/interactive/src/plan/mod.rs b/interactive/src/plan/mod.rs index deff18a4a..31d99041d 100644 --- a/interactive/src/plan/mod.rs +++ b/interactive/src/plan/mod.rs @@ -169,7 +169,11 @@ impl Render for Plan { input_arrangement }; - let output = input.reduce_abelian::<_,KeyBuilder<_,_,_>,KeySpine<_,_,_>>("Distinct", move |_,_,t| t.push(((), 1))); + let output = input.reduce_abelian::<_,KeyBuilder<_,_,_>,KeySpine<_,_,_>,_>( + "Distinct", + move |_,_,t| t.push(((), 1)), + |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, + ); arrangements.set_unkeyed(&self, &output.trace); output.as_collection(|k,&()| k.clone()) From 1f604d09d398c59fbdb70e94f528326994e70f4b Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 9 Apr 2026 17:15:46 +0200 Subject: [PATCH 20/27] Remove TimelyStack and all dependent types (#715) Remove TimelyStack, TStack layout, ColumnationChunker, ColInternalMerger, and all Col* type aliases (ColValSpine, ColKeySpine, ColValBuilder, etc.) from the codebase. The columnation crate dependency is retained. Co-authored-by: Claude Opus 4.6 (1M context) --- differential-dataflow/examples/spines.rs | 7 - differential-dataflow/src/consolidation.rs | 33 -- differential-dataflow/src/containers.rs | 302 ------------------ differential-dataflow/src/lib.rs | 1 - .../src/trace/implementations/chunker.rs | 111 +------ .../trace/implementations/merge_batcher.rs | 125 +------- .../src/trace/implementations/mod.rs | 114 ------- .../src/trace/implementations/ord_neu.rs | 24 +- .../src/trace/implementations/rhh.rs | 17 +- 9 files changed, 12 insertions(+), 722 deletions(-) delete mode 100644 differential-dataflow/src/containers.rs diff --git a/differential-dataflow/examples/spines.rs b/differential-dataflow/examples/spines.rs index a72a6f34b..b2e9bf5dd 100644 --- a/differential-dataflow/examples/spines.rs +++ b/differential-dataflow/examples/spines.rs @@ -32,13 +32,6 @@ fn main() { let (keys_input, keys) = scope.new_collection::(); 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<_,_,_>>(); - keys.join_core(data, |_k, &(), &()| Option::<()>::None) - .probe_with(&mut probe); - }, "old" => { use differential_dataflow::trace::implementations::ord_neu::{OrdKeyBatcher, RcOrdKeyBuilder, OrdKeySpine}; let data = data.arrange::, RcOrdKeyBuilder<_,_,_>, OrdKeySpine<_,_,_>>(); diff --git a/differential-dataflow/src/consolidation.rs b/differential-dataflow/src/consolidation.rs index 43d403d38..a56a42530 100644 --- a/differential-dataflow/src/consolidation.rs +++ b/differential-dataflow/src/consolidation.rs @@ -11,7 +11,6 @@ //! specific behavior you require. use std::collections::VecDeque; -use columnation::Columnation; use timely::container::{ContainerBuilder, PushInto}; use crate::Data; use crate::difference::Semigroup; @@ -260,38 +259,6 @@ impl Consolidate for Vec<(D, T, R)> { } } -impl Consolidate for crate::containers::TimelyStack<(D, T, R)> { - fn len(&self) -> usize { self[..].len() } - fn clear(&mut self) { crate::containers::TimelyStack::clear(self) } - fn consolidate_into(&mut self, target: &mut Self) { - let len = self[..].len(); - let mut indices: Vec = (0..len).collect(); - indices.sort_unstable_by(|&i, &j| { - let (d1, t1, _) = &self[i]; - let (d2, t2, _) = &self[j]; - (d1, t1).cmp(&(d2, t2)) - }); - target.clear(); - let mut idx = 0; - while idx < indices.len() { - let (d, t, r) = &self[indices[idx]]; - let mut r_owned = r.clone(); - idx += 1; - while idx < indices.len() { - let (d2, t2, r2) = &self[indices[idx]]; - if d == d2 && t == t2 { - r_owned.plus_equals(r2); - idx += 1; - } else { break; } - } - if !r_owned.is_zero() { - target.copy_destructured(d, t, &r_owned); - } - } - self.clear(); - } -} - #[cfg(test)] mod tests { use super::*; 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/lib.rs b/differential-dataflow/src/lib.rs index b5561cfb2..0bf44e7ae 100644 --- a/differential-dataflow/src/lib.rs +++ b/differential-dataflow/src/lib.rs @@ -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/trace/implementations/chunker.rs b/differential-dataflow/src/trace/implementations/chunker.rs index 93a9e516f..4737838f2 100644 --- a/differential-dataflow/src/trace/implementations/chunker.rs +++ b/differential-dataflow/src/trace/implementations/chunker.rs @@ -2,119 +2,10 @@ 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, Consolidate}; -use crate::difference::Semigroup; - -/// Chunk a stream of vectors into chains of columnation stacks. -/// -/// This chunker accumulates into a `Vec` (not a `TimelyStack`) for efficient -/// in-place sorting and consolidation, then copies the consolidated results -/// into `TimelyStack` chunks. This avoids the cost of sorting through -/// columnation indirection. -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 - } - } - - 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)>) { - 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() - } -} +use crate::consolidation::Consolidate; /// Chunk a stream of containers into chains of vectors. pub struct ContainerChunker { diff --git a/differential-dataflow/src/trace/implementations/merge_batcher.rs b/differential-dataflow/src/trace/implementations/merge_batcher.rs index 1877c8825..d7656fd0e 100644 --- a/differential-dataflow/src/trace/implementations/merge_batcher.rs +++ b/differential-dataflow/src/trace/implementations/merge_batcher.rs @@ -277,11 +277,11 @@ pub mod container { /// calling `extract` again. The caller invokes `extract` repeatedly /// until `*position >= self.len()`. /// - /// This shape exists because `at_capacity()` for `Vec` and - /// `TimelyStack` is `len() == capacity()`, which silently becomes - /// false again the moment a push past capacity grows the backing - /// allocation. Without per-element yielding, a single `extract` call - /// can quietly produce oversized output chunks. + /// This shape exists because `at_capacity()` for `Vec` is + /// `len() == capacity()`, which silently becomes false again the + /// moment a push past capacity grows the backing allocation. + /// Without per-element yielding, a single `extract` call can + /// quietly produce oversized output chunks. fn extract( &mut self, position: &mut usize, @@ -294,9 +294,6 @@ pub mod container { /// A `Merger` for `Vec` containers, which contain owned data and need special treatment. pub type VecInternalMerger = VecMerger; - /// A `Merger` using internal iteration for `TimelyStack` containers. - pub type ColInternalMerger = InternalMerger>; - /// A `Merger` implementation for `Vec<(D, T, R)>` that drains owned inputs. pub struct VecMerger { _marker: PhantomData<(D, T, R)>, @@ -683,116 +680,4 @@ pub mod container { } } - /// Implementation of `InternalMerge` for `TimelyStack<(D, T, R)>`. - pub mod columnation_internal { - use std::cmp::Ordering; - use columnation::Columnation; - use timely::PartialOrder; - use timely::container::SizableContainer; - use timely::progress::frontier::{Antichain, AntichainRef}; - use crate::containers::TimelyStack; - use crate::difference::Semigroup; - use super::InternalMerge; - - impl InternalMerge for TimelyStack<(D, T, R)> - where - D: Ord + Columnation + Clone + 'static, - T: Ord + Columnation + Clone + PartialOrder + 'static, - R: Default + Semigroup + Columnation + Clone + 'static, - { - type TimeOwned = T; - - fn len(&self) -> usize { self[..].len() } - fn clear(&mut self) { TimelyStack::clear(self) } - - 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) - } - - fn merge_from( - &mut self, - others: &mut [Self], - positions: &mut [usize], - ) { - match others.len() { - 0 => {}, - 1 => { - let other = &mut others[0]; - let pos = &mut positions[0]; - if self[..].is_empty() && *pos == 0 { - std::mem::swap(self, other); - return; - } - for i in *pos .. other[..].len() { - self.copy(&other[i]); - } - *pos = other[..].len(); - }, - 2 => { - let (left, right) = others.split_at_mut(1); - let other1 = &left[0]; - let other2 = &right[0]; - - let mut stash = R::default(); - - while positions[0] < other1[..].len() && positions[1] < other2[..].len() && !self.at_capacity() { - let (d1, t1, _) = &other1[positions[0]]; - let (d2, t2, _) = &other2[positions[1]]; - match (d1, t1).cmp(&(d2, t2)) { - Ordering::Less => { - self.copy(&other1[positions[0]]); - positions[0] += 1; - } - Ordering::Greater => { - self.copy(&other2[positions[1]]); - positions[1] += 1; - } - Ordering::Equal => { - let (_, _, r1) = &other1[positions[0]]; - let (_, _, r2) = &other2[positions[1]]; - stash.clone_from(r1); - stash.plus_equals(r2); - if !stash.is_zero() { - let (d, t, _) = &other1[positions[0]]; - self.copy_destructured(d, t, &stash); - } - positions[0] += 1; - positions[1] += 1; - } - } - } - }, - n => unimplemented!("{n}-way merge not yet supported"), - } - } - - fn extract( - &mut self, - position: &mut usize, - upper: AntichainRef, - frontier: &mut Antichain, - keep: &mut Self, - ship: &mut Self, - ) { - let len = self[..].len(); - while *position < len && !keep.at_capacity() && !ship.at_capacity() { - let (data, time, diff) = &self[*position]; - if upper.less_equal(time) { - frontier.insert_with(time, |time| time.clone()); - keep.copy_destructured(data, time, diff); - } else { - ship.copy_destructured(data, time, diff); - } - *position += 1; - } - } - } - } } diff --git a/differential-dataflow/src/trace/implementations/mod.rs b/differential-dataflow/src/trace/implementations/mod.rs index 082af6812..836c6f612 100644 --- a/differential-dataflow/src/trace/implementations/mod.rs +++ b/differential-dataflow/src/trace/implementations/mod.rs @@ -57,12 +57,10 @@ 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 crate::containers::TimelyStack; use crate::lattice::Lattice; use crate::difference::Semigroup; @@ -223,27 +221,6 @@ impl> Layout for Vector { 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 { @@ -447,72 +424,13 @@ where } } -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}; /// 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<'_>`. @@ -650,38 +568,6 @@ pub mod containers { } } - // The `ToOwned` requirement exists to satisfy `self.reserve_items`, who must for now - // be presented with the actual contained type, rather than a type that borrows into it. - impl BatchContainer for TimelyStack { - type Owned = T; - type ReadItem<'a> = &'a T; - - #[inline(always)] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { item.clone() } - #[inline(always)] fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.clone_from(item); } - - fn reborrow<'b, 'a: 'b>(item: Self::ReadItem<'a>) -> Self::ReadItem<'b> { item } - - fn push_ref(&mut self, item: Self::ReadItem<'_>) { self.push_into(item) } - fn push_own(&mut self, item: &Self::Owned) { self.push_into(item) } - - fn clear(&mut self) { self.clear() } - - fn with_capacity(size: usize) -> Self { - Self::with_capacity(size) - } - fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { - let mut new = Self::default(); - new.reserve_regions(std::iter::once(cont1).chain(std::iter::once(cont2))); - new - } - fn index(&self, index: usize) -> Self::ReadItem<'_> { - &self[index] - } - fn len(&self) -> usize { - self[..].len() - } - } - /// A container that accepts slices `[B::Item]`. pub struct SliceContainer { /// Offsets that bound each contained slice. diff --git a/differential-dataflow/src/trace/implementations/ord_neu.rs b/differential-dataflow/src/trace/implementations/ord_neu.rs index 14bc8a2db..0d5315f6e 100644 --- a/differential-dataflow/src/trace/implementations/ord_neu.rs +++ b/differential-dataflow/src/trace/implementations/ord_neu.rs @@ -10,14 +10,13 @@ use std::rc::Rc; -use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, ContainerChunker}; +use crate::trace::implementations::chunker::ContainerChunker; use crate::trace::implementations::spine_fueled::Spine; use crate::trace::implementations::merge_batcher::MergeBatcher; -use crate::trace::implementations::merge_batcher::container::{VecInternalMerger, ColInternalMerger}; +use crate::trace::implementations::merge_batcher::container::VecInternalMerger; use crate::trace::rc_blanket_impls::RcBuilder; -use super::{Layout, Vector, TStack}; +use super::{Layout, Vector}; pub use self::val_batch::{OrdValBatch, OrdValBuilder}; pub use self::key_batch::{OrdKeyBatch, OrdKeyBuilder}; @@ -32,13 +31,6 @@ pub type RcOrdValBuilder = RcBuilder = 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)>, ColInternalMerger<(K,V),T,R>>; -/// A builder for columnar storage. -pub type ColValBuilder = RcBuilder, TimelyStack<((K,V),T,R)>>>; - /// A trace implementation using a spine of ordered lists. pub type OrdKeySpine = Spine>>>; /// A batcher for ordered lists. @@ -49,16 +41,6 @@ pub type RcOrdKeyBuilder = RcBuilder // /// 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)>, ColInternalMerger<(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 use layers::{Vals, Upds}; /// Layers are containers of lists of some type. /// diff --git a/differential-dataflow/src/trace/implementations/rhh.rs b/differential-dataflow/src/trace/implementations/rhh.rs index bf2f252a9..7308b3a49 100644 --- a/differential-dataflow/src/trace/implementations/rhh.rs +++ b/differential-dataflow/src/trace/implementations/rhh.rs @@ -11,14 +11,13 @@ use std::cmp::Ordering; use serde::{Deserialize, Serialize}; use crate::Hashable; -use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, ContainerChunker}; +use crate::trace::implementations::chunker::ContainerChunker; use crate::trace::implementations::merge_batcher::MergeBatcher; -use crate::trace::implementations::merge_batcher::container::{VecInternalMerger, ColInternalMerger}; +use crate::trace::implementations::merge_batcher::container::VecInternalMerger; use crate::trace::implementations::spine_fueled::Spine; use crate::trace::rc_blanket_impls::RcBuilder; -use super::{Layout, Vector, TStack}; +use super::{Layout, Vector}; use self::val_batch::{RhhValBatch, RhhValBuilder}; @@ -32,16 +31,6 @@ pub type VecBuilder = RcBuilder, Vec< // /// 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)>, ColInternalMerger<(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>>>; - /// A carrier trait indicating that the type's `Ord` and `PartialOrd` implementations are by `Hashable::hashed()`. pub trait HashOrdered: Hashable { } From 954c2ea3a3c9976fd0b8df5499043f1167f871b9 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Thu, 9 Apr 2026 14:45:16 -0400 Subject: [PATCH 21/27] Track timely `master` changes, around `Child` scope changes (#714) * Track timely's Child changes * Tidy bounds * Absorb next wave of changes * Remove unused, and Arranged impls * Remove timestamp generic from Arranged * Standardize T for timestamps, Tr for traces * Correct local timely reference * Convert G generics to T * Remove as Timestamp prompts * Remove T generics bound by Tr * Remove Scope::clone calls * Further tightening of traits * Correct docs --- Cargo.toml | 4 +- diagnostics/src/logging.rs | 56 ++-- differential-dataflow/examples/accumulate.rs | 9 +- differential-dataflow/examples/bfs.rs | 5 +- .../examples/columnar/columnar_support.rs | 26 +- .../examples/columnar/main.rs | 25 +- differential-dataflow/examples/dynamic.rs | 10 +- differential-dataflow/examples/graspan.rs | 31 +-- differential-dataflow/examples/interpreted.rs | 5 +- .../examples/iterate_container.rs | 6 +- differential-dataflow/examples/monoid-bfs.rs | 10 +- differential-dataflow/examples/pagerank.rs | 9 +- differential-dataflow/examples/progress.rs | 9 +- .../examples/stackoverflow.rs | 5 +- .../src/algorithms/graphs/bfs.rs | 9 +- .../src/algorithms/graphs/bijkstra.rs | 24 +- .../src/algorithms/graphs/propagate.rs | 20 +- .../src/algorithms/graphs/scc.rs | 22 +- .../src/algorithms/graphs/sequential.rs | 16 +- .../src/algorithms/identifiers.rs | 12 +- .../src/algorithms/prefix_sum.rs | 26 +- differential-dataflow/src/capture.rs | 25 +- differential-dataflow/src/collection.rs | 256 +++++++++--------- differential-dataflow/src/dynamic/mod.rs | 4 +- differential-dataflow/src/input.rs | 29 +- differential-dataflow/src/logging.rs | 3 +- .../src/operators/arrange/agent.rs | 23 +- .../src/operators/arrange/arrangement.rs | 142 +++++----- .../src/operators/arrange/upsert.rs | 28 +- differential-dataflow/src/operators/count.rs | 37 ++- .../src/operators/iterate.rs | 58 ++-- differential-dataflow/src/operators/join.rs | 16 +- differential-dataflow/src/operators/reduce.rs | 58 ++-- .../src/operators/threshold.rs | 43 ++- differential-dataflow/tests/bfs.rs | 5 +- differential-dataflow/tests/scc.rs | 13 +- dogsdogsdogs/examples/delta_query.rs | 5 +- dogsdogsdogs/examples/delta_query2.rs | 1 - dogsdogsdogs/examples/delta_query_wcoj.rs | 3 +- dogsdogsdogs/examples/ngo.rs | 5 +- dogsdogsdogs/src/altneu.rs | 2 +- dogsdogsdogs/src/calculus.rs | 24 +- dogsdogsdogs/src/lib.rs | 45 ++- dogsdogsdogs/src/operators/count.rs | 11 +- dogsdogsdogs/src/operators/half_join.rs | 54 ++-- dogsdogsdogs/src/operators/half_join2.rs | 43 ++- dogsdogsdogs/src/operators/lookup_map.rs | 14 +- dogsdogsdogs/src/operators/propose.rs | 20 +- dogsdogsdogs/src/operators/validate.rs | 11 +- experiments/Cargo.toml | 2 +- experiments/src/bin/deals-interactive.rs | 8 +- experiments/src/bin/deals.rs | 19 +- experiments/src/bin/graphs-interactive-alt.rs | 34 +-- .../src/bin/graphs-interactive-neu-zwei.rs | 26 +- experiments/src/bin/graphs-interactive-neu.rs | 29 +- experiments/src/bin/graphs-interactive.rs | 29 +- experiments/src/bin/graphs-static.rs | 42 +-- experiments/src/bin/graphs.rs | 18 +- experiments/src/bin/graspan1.rs | 6 +- experiments/src/bin/graspan2.rs | 29 +- interactive/src/command.rs | 3 +- interactive/src/logging.rs | 11 +- interactive/src/manager.rs | 3 +- interactive/src/plan/concat.rs | 6 +- interactive/src/plan/filter.rs | 8 +- interactive/src/plan/join.rs | 8 +- interactive/src/plan/map.rs | 8 +- interactive/src/plan/mod.rs | 17 +- interactive/src/plan/sfw.rs | 12 +- mdbook/src/chapter_2/chapter_2_1.md | 10 +- mdbook/src/chapter_2/chapter_2_2.md | 5 +- mdbook/src/chapter_2/chapter_2_3.md | 5 +- mdbook/src/chapter_2/chapter_2_4.md | 10 +- mdbook/src/chapter_2/chapter_2_5.md | 5 +- mdbook/src/chapter_2/chapter_2_6.md | 5 +- mdbook/src/chapter_2/chapter_2_7.md | 23 +- server/Cargo.toml | 2 +- server/src/lib.rs | 8 +- tpchlike/src/lib.rs | 2 +- 79 files changed, 818 insertions(+), 892 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d9d402c79..895c0407e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,9 +23,9 @@ rust-version = "1.86" [workspace.dependencies] differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.22.0" } -timely = { version = "0.28", default-features = false } +#timely = { version = "0.28", default-features = false } columnar = { version = "0.12", default-features = false } -#timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } +timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } #timely = { path = "../timely-dataflow/timely/", default-features = false } [workspace.lints.clippy] diff --git a/diagnostics/src/logging.rs b/diagnostics/src/logging.rs index b47721e62..f0fac8d2b 100644 --- a/diagnostics/src/logging.rs +++ b/diagnostics/src/logging.rs @@ -34,7 +34,6 @@ use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::trace::implementations::{KeySpine, ValSpine}; use differential_dataflow::{AsCollection, VecCollection}; -use timely::communication::Allocate; use timely::container::CapacityContainerBuilder; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::capture::{Event, EventLink, Replay, Capture}; @@ -244,12 +243,11 @@ fn quantize(time: Duration, interval: Duration) -> Duration { } /// Quantize timestamps in a collection's inner stream. -fn quantize_collection( - collection: VecCollection, +fn quantize_collection( + collection: VecCollection, interval: Duration, -) -> VecCollection +) -> VecCollection where - S: Scope, D: differential_dataflow::Data, { collection @@ -276,7 +274,7 @@ where /// /// Returns a [`LoggingState`] with trace handles and a [`SinkHandle`] for /// the WebSocket thread. -pub fn register(worker: &mut Worker, log_logging: bool) -> LoggingState { +pub fn register(worker: &mut Worker, log_logging: bool) -> LoggingState { let start = Instant::now(); // Event links for logging capture (worker-internal, Rc-based). @@ -377,8 +375,8 @@ pub fn register(worker: &mut Worker, log_logging: bool) -> Loggi } } -fn install_loggers( - worker: &mut Worker, +fn install_loggers( + worker: &mut Worker, t_link: Rc>>, d_link: Rc>>, ) { @@ -402,11 +400,11 @@ fn install_loggers( // ============================================================================ /// Internal: collections before arrangement, used for the cross-join. -struct TimelyCollections { - operators: VecCollection), i64>, - channels: VecCollection, (usize, usize), (usize, usize)), i64>, - elapsed: VecCollection, - messages: VecCollection, +struct TimelyCollections { + operators: VecCollection), i64>, + channels: VecCollection, (usize, usize), (usize, usize)), i64>, + elapsed: VecCollection, + messages: VecCollection, } #[derive(Default)] @@ -416,10 +414,10 @@ struct TimelyDemuxState { } /// Build timely logging collections and arrangements. -fn construct_timely>( - scope: &mut S, - stream: Stream>, -) -> (TimelyTraces, TimelyCollections) { +fn construct_timely( + scope: &mut Scope, + stream: Stream>, +) -> (TimelyTraces, TimelyCollections) { type OpUpdate = ((usize, String, Vec), Duration, i64); type ChUpdate = ((usize, Vec, (usize, usize), (usize, usize)), Duration, i64); type ElUpdate = (usize, Duration, i64); @@ -536,21 +534,21 @@ fn construct_timely>( // ============================================================================ /// Internal: collections before arrangement, used for the cross-join. -struct DifferentialCollections { - arrangement_batches: VecCollection, - arrangement_records: VecCollection, - sharing: VecCollection, - batcher_records: VecCollection, - batcher_size: VecCollection, - batcher_capacity: VecCollection, - batcher_allocations: VecCollection, +struct DifferentialCollections { + arrangement_batches: VecCollection, + arrangement_records: VecCollection, + sharing: VecCollection, + batcher_records: VecCollection, + batcher_size: VecCollection, + batcher_capacity: VecCollection, + batcher_allocations: VecCollection, } /// Build differential logging collections and arrangements. -fn construct_differential>( - scope: &mut S, - stream: Stream>, -) -> (DifferentialTraces, DifferentialCollections) { +fn construct_differential( + scope: &mut Scope, + stream: Stream>, +) -> (DifferentialTraces, DifferentialCollections) { type Update = (usize, Duration, i64); let mut demux = OperatorBuilder::new("Differential Demux".to_string(), scope.clone()); diff --git a/differential-dataflow/examples/accumulate.rs b/differential-dataflow/examples/accumulate.rs index 0fa792926..d8da7bdcd 100644 --- a/differential-dataflow/examples/accumulate.rs +++ b/differential-dataflow/examples/accumulate.rs @@ -7,7 +7,7 @@ fn main() { let keys: usize = std::env::args().nth(1).unwrap().parse().unwrap(); let batch: usize = 10_000; - // This computation demonstrates in-place accumulation of arbitrarily large + // This computation demonstrates in-place accumulation of arbitrarily large // volumes of input data, consuming space bounded by the number of distinct keys. timely::execute_from_args(std::env::args().skip(2), move |worker| { @@ -17,11 +17,10 @@ fn main() { let mut input = worker.dataflow::<(), _, _>(|scope| { let (input, data) = scope.new_collection::<_, isize>(); - use timely::dataflow::Scope; scope.iterative::(|inner| { data.enter_at(inner, |_| 0) .consolidate() - .leave() + .leave(&scope) }); input @@ -41,7 +40,7 @@ fn main() { } counter += batch; - worker.step(); + worker.step(); let elapsed = timer.elapsed(); if elapsed.as_secs() as usize > last_sec { @@ -54,4 +53,4 @@ fn main() { } }).unwrap(); -} \ No newline at end of file +} diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index af875d5d5..db97f3938 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -1,6 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -91,9 +90,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index 8b6e10543..03a260218 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -592,7 +592,7 @@ pub mod arrangement { impl Merger for TrieMerger where - U::Time: Ord + PartialOrder + Clone + 'static, + U::Time: 'static, { type Chunk = Updates; type Time = U::Time; @@ -646,7 +646,7 @@ pub mod arrangement { impl TrieMerger where - U::Time: Ord + PartialOrder + Clone + 'static, + U::Time: 'static, { /// Iterator-based merge: flatten, merge, consolidate, form. /// Correct but slow — used as fallback. @@ -1839,13 +1839,12 @@ pub mod updates { /// joins output times with input times, multiplies output diffs with input diffs. /// /// This subsumes map, filter, negate, and enter_at for columnar collections. -pub fn join_function( - input: differential_dataflow::Collection>, +pub fn join_function( + input: differential_dataflow::Collection>, mut logic: L, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, - G::Timestamp: differential_dataflow::lattice::Lattice, + U::Time: differential_dataflow::lattice::Lattice, U: layout::ColumnarUpdate>, I: IntoIterator, L: FnMut( @@ -1893,12 +1892,11 @@ type DynTime = timely::order::Product( - input: differential_dataflow::Collection>, +pub fn leave_dynamic( + input: differential_dataflow::Collection>, level: usize, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, K: columnar::Columnar, V: columnar::Columnar, R: columnar::Columnar, @@ -1972,14 +1970,12 @@ where /// /// Cursors through each batch and pushes `(key, val, time, diff)` refs into /// a `ValColBuilder`, which sorts and consolidates on flush. -pub fn as_recorded_updates( +pub fn as_recorded_updates( arranged: differential_dataflow::operators::arrange::Arranged< - G, differential_dataflow::operators::arrange::TraceAgent>, >, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, U: layout::ColumnarUpdate, { use timely::dataflow::operators::generic::Operator; diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs index 85b573157..a6a2b9ccd 100644 --- a/differential-dataflow/examples/columnar/main.rs +++ b/differential-dataflow/examples/columnar/main.rs @@ -93,7 +93,6 @@ fn main() { mod reachability { use timely::order::Product; - use timely::dataflow::Scope; use differential_dataflow::Collection; use differential_dataflow::AsCollection; use differential_dataflow::operators::iterate::Variable; @@ -110,14 +109,14 @@ mod reachability { /// Compute the set of nodes reachable from `roots` along directed `edges`. /// /// Returns `(node, ())` for each reachable node. - pub fn reach>( - edges: Collection>, - roots: Collection>, - ) -> Collection> + pub fn reach( + edges: Collection>, + roots: Collection>, + ) -> Collection> { - let mut scope = edges.inner.scope(); + let outer = edges.inner.scope(); - scope.iterative::(|nested| { + outer.iterative::(|nested| { let summary = Product::new(Time::default(), 1); let roots_inner = roots.enter(nested); @@ -128,13 +127,13 @@ mod reachability { let edges_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; let reach_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let edges_arr = arrange_core::<_, _, + let edges_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, >(edges_inner.inner, edges_pact, "Edges"); - let reach_arr = arrange_core::<_, _, + let reach_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, @@ -142,7 +141,7 @@ mod reachability { // join_traces with ValColBuilder: produces Stream<_, RecordedUpdates<...>>. let proposed = - join_traces::<_, _, _, _, ValColBuilder<(Node, (), IterTime, Diff)>>( + join_traces::<_, _, _, ValColBuilder<(Node, (), IterTime, Diff)>>( edges_arr, reach_arr, |_src, dst, (), time, d1, d2, session| { @@ -158,7 +157,7 @@ mod reachability { // Arrange for reduce. let combined_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let combined_arr = arrange_core::<_, _, + let combined_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, @@ -181,12 +180,12 @@ mod reachability { }); // Extract RecordedUpdates from the Arranged's batch stream. - let result_col = as_recorded_updates::<_, (Node, (), IterTime, Diff)>(result); + let result_col = as_recorded_updates::<(Node, (), IterTime, Diff)>(result); variable.set(result_col.clone()); // Leave the iterative scope. - result_col.leave() + result_col.leave(&outer) }) } } diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index a844c28bb..c0206fbde 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -1,6 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -91,9 +90,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { use timely::order::Product; use iterate::Variable; @@ -103,7 +102,8 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.scope().iterative::, _, _>(|inner| { + let outer = nodes.scope(); + outer.iterative::, _, _>(|inner| { // These enter the statically bound scope, rather than any iterative scopes. // We do not *need* to enter them into the dynamic scope, as they are static @@ -126,7 +126,7 @@ where // Leave the dynamic iteration, stripping off the last timestamp coordinate. next.leave_dynamic(1) .inspect(|x| println!("{:?}", x)) - .leave() + .leave(&outer) }) } diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index 79c7fd184..4725cf4af 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -5,7 +5,6 @@ use std::fs::File; use timely::progress::Timestamp; use timely::order::Product; use timely::dataflow::Scope; -use timely::dataflow::scopes::ScopeParent; use differential_dataflow::VecCollection; use differential_dataflow::lattice::Lattice; @@ -69,7 +68,7 @@ use differential_dataflow::operators::arrange::{Arranged, TraceAgent}; type TraceKeyHandle = TraceAgent>; type TraceValHandle = TraceAgent>; -type Arrange = Arranged::Timestamp, R>>; +type Arrange = Arranged>; /// An evolving set of edges. /// @@ -79,17 +78,17 @@ 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> { - variable: VecVariable, - collection: VecCollection, - current: VecCollection, - forward: Option>, - reverse: Option>, +pub struct EdgeVariable { + variable: VecVariable, + collection: VecCollection, + current: VecCollection, + forward: Option>, + reverse: Option>, } -impl> EdgeVariable { +impl EdgeVariable { /// Creates a new variable initialized with `source`. - pub fn from(source: VecCollection, step: ::Summary) -> Self { + pub fn from(source: VecCollection, step: T::Summary) -> Self { let (variable, collection) = VecVariable::new(&mut source.scope(), step); EdgeVariable { variable, @@ -100,7 +99,7 @@ impl> EdgeVariable { } } /// Concatenates `production` into the definition of the variable. - pub fn add_production(&mut self, production: VecCollection) { + pub fn add_production(&mut self, production: VecCollection) { self.current = self.current.clone().concat(production); } /// Finalizes the variable, connecting its recursive definition. @@ -113,14 +112,14 @@ impl> EdgeVariable { self.variable.set(distinct); } /// The collection arranged in the forward direction. - pub fn forward(&mut self) -> &Arrange { + pub fn forward(&mut self) -> &Arrange { if self.forward.is_none() { self.forward = Some(self.collection.clone().arrange_by_key()); } self.forward.as_ref().unwrap() } /// The collection arranged in the reverse direction. - pub fn reverse(&mut self) -> &Arrange { + pub fn reverse(&mut self) -> &Arrange { if self.reverse.is_none() { self.reverse = Some(self.collection.clone().map(|(x,y)| (y,x)).arrange_by_key()); } @@ -151,9 +150,9 @@ impl Query { } /// Creates a dataflow implementing the query, and returns input and trace handles. - pub fn render_in(&self, scope: &mut G) -> BTreeMap> + pub fn render_in(&self, scope: &mut Scope) -> BTreeMap> where - G: Scope, + T: Timestamp + Lattice + ::timely::order::TotalOrder, { // Create new input (handle, stream) pairs let mut input_map = BTreeMap::new(); @@ -171,7 +170,7 @@ impl Query { // create variables and result handles for each named relation. for (name, (input, collection)) in input_map { let edge_variable = EdgeVariable::from(collection.enter(subscope), Product::new(Default::default(), 1)); - let trace = edge_variable.collection.clone().leave().arrange_by_self().trace; + let trace = edge_variable.collection.clone().leave(&scope).arrange_by_self().trace; result_map.insert(name.clone(), RelationHandles { input, trace }); variable_map.insert(name.clone(), edge_variable); } diff --git a/differential-dataflow/examples/interpreted.rs b/differential-dataflow/examples/interpreted.rs index 4c6e3429c..31a622108 100644 --- a/differential-dataflow/examples/interpreted.rs +++ b/differential-dataflow/examples/interpreted.rs @@ -1,5 +1,4 @@ use std::hash::Hash; -use timely::dataflow::*; use timely::dataflow::operators::*; use differential_dataflow::VecCollection; @@ -37,9 +36,9 @@ fn main() { }).unwrap(); } -fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> +fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> where - G: Scope, + T: timely::progress::Timestamp + Lattice + Hash + Ord, { // arrange the edge relation three ways. diff --git a/differential-dataflow/examples/iterate_container.rs b/differential-dataflow/examples/iterate_container.rs index f7ef3764e..d2fa85302 100644 --- a/differential-dataflow/examples/iterate_container.rs +++ b/differential-dataflow/examples/iterate_container.rs @@ -3,7 +3,7 @@ use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Operator; use timely::order::Product; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; use differential_dataflow::{AsCollection, Collection}; use differential_dataflow::input::Input; @@ -33,7 +33,7 @@ impl, TS> ResultsIn for ContainerWrapper { #[inline(always)] fn results_in(self, step: &TS) -> Self { ContainerWrapper(self.0.results_in(step)) } } -fn wrap(stream: Stream) -> Stream> { +fn wrap(stream: Stream) -> Stream> { let mut builder = OperatorBuilder::new("Wrap".to_string(), stream.scope()); let (mut output, stream_out) = builder.new_output(); let mut input = builder.new_input(stream, Pipeline); @@ -77,7 +77,7 @@ fn main() { }).as_collection().consolidate(); let result = wrap(result.inner).as_collection(); variable.set(result); - collection.leave() + collection.leave(&scope) }); }) } diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index cce06674e..f658cce0f 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -1,7 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; use serde::{Deserialize, Serialize}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -122,12 +121,13 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // repeatedly update minimal distances each node can be reached from each root - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { use differential_dataflow::operators::iterate::Variable; use differential_dataflow::trace::implementations::{KeySpine, KeyBuilder}; @@ -152,6 +152,6 @@ where .as_collection(|k,()| *k); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index d779ed36e..b649316ad 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -77,9 +77,9 @@ fn main() { // Returns a weighted collection in which the weight of each node is proportional // to its PageRank in the input graph `edges`. -fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection +fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice, { // initialize many surfers at each node. let nodes = @@ -92,7 +92,8 @@ where .map(|(src,_dst)| src) .count(); - edges.scope().iterative::(|inner| { + let outer = edges.scope(); + outer.iterative::(|inner| { // Bring various collections into the scope. let edges = edges.enter(inner); @@ -130,6 +131,6 @@ where // Bind the recursive variable, return its limit. ranks_bind.set(pushed.clone()); - pushed.leave() + pushed.leave(&outer) }) } diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 634e43d56..0229263d8 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -1,7 +1,6 @@ //! A demonstration of timely dataflow progress tracking, using differential dataflow operators. use timely::PartialOrder; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -120,7 +119,7 @@ fn frontier( times: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp+std::hash::Hash, { // Translate node and edge transitions into a common Location to Location edge with an associated Summary. @@ -154,7 +153,7 @@ fn summarize( edges: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp, { // Start from trivial reachability from each input to itself. @@ -194,12 +193,12 @@ where /// Identifies cycles along paths that do not increment timestamps. -fn find_cycles( +fn find_cycles( nodes: VecCollection, edges: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp, { // Retain node connections along "default" timestamp summaries. diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index 3fe521ee2..5d25b19b1 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::InputSession; @@ -105,9 +104,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // 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..a9d5ed307 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -2,16 +2,16 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::operators::*; use crate::lattice::Lattice; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord, N: ExchangeData+Hash, { let edges = edges.arrange_by_key(); @@ -22,9 +22,8 @@ use crate::trace::TraceReader; use crate::operators::arrange::Arranged; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs_arranged(edges: Arranged, roots: VecCollection) -> VecCollection +pub fn bfs_arranged(edges: Arranged, roots: VecCollection) -> VecCollection where - G: Scope, N: ExchangeData+Hash, Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, { diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index 9c61716ef..82f47ecec 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -3,7 +3,7 @@ use std::hash::Hash; use timely::order::Product; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -19,9 +19,9 @@ use crate::operators::iterate::Variable; /// Goals that cannot reach from the source to the target are relatively expensive, as /// the entire graph must be explored to confirm this. A graph connectivity pre-filter /// could be good insurance here. -pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection +pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord, N: ExchangeData+Hash, { let forward = edges.clone().arrange_by_key(); @@ -33,19 +33,17 @@ use crate::trace::TraceReader; use crate::operators::arrange::Arranged; /// Bi-directional Dijkstra search using arranged forward and reverse edge collections. -pub fn bidijkstra_arranged( - forward: Arranged, - reverse: Arranged, - goals: VecCollection -) -> VecCollection +pub fn bidijkstra_arranged( + forward: Arranged, + reverse: Arranged, + goals: VecCollection +) -> VecCollection where - G: Scope, N: ExchangeData+Hash, Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, { - forward - .stream - .scope().iterative::(|inner| { + let outer = forward.stream.scope(); + outer.iterative::(|inner| { let forward_edges = forward.enter(inner); let reverse_edges = reverse.enter(inner); @@ -120,6 +118,6 @@ where reverse_bind.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 2728de6e2..54efb079d 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -2,7 +2,7 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -13,9 +13,9 @@ use crate::difference::{Abelian, Multiply}; /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection +pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -30,9 +30,9 @@ where /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection +pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -51,15 +51,14 @@ use crate::operators::arrange::arrangement::Arranged; /// This variant takes a pre-arranged edge collection, to facilitate re-use, and allows /// a method `logic` to specify the rounds in which we introduce various labels. The output /// of `logic should be a number in the interval \[0,64\], -pub fn propagate_core(edges: Arranged, nodes: VecCollection, logic: F) -> VecCollection +pub fn propagate_core(edges: Arranged, nodes: VecCollection, logic: F) -> VecCollection where - G: Scope, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, R: From, L: ExchangeData, - Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Time:Hash, Diff=R>+Clone+'static, + Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=R, Time: Hash>+Clone+'static, F: Fn(&L)->u64+Clone+'static, { // Morally the code performs the following iterative computation. However, in the interest of a simplified @@ -78,7 +77,8 @@ where // }) use timely::order::Product; - nodes.scope().scoped::,_,_>("Propagate", |scope| { + let outer = nodes.scope(); + outer.scoped::,_,_>("Propagate", |scope| { use crate::operators::iterate::Variable; use crate::trace::implementations::{ValBuilder, ValSpine}; @@ -104,6 +104,6 @@ where labels .as_collection(|k,v| (k.clone(), v.clone())) - .leave() + .leave(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index b730302e7..f158cdc08 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -3,7 +3,7 @@ use std::mem; use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -12,16 +12,17 @@ use crate::difference::{Abelian, Multiply}; use super::propagate::propagate; /// Returns the subset of edges in the same strongly connected component. -pub fn strongly_connected(graph: VecCollection) -> VecCollection +pub fn strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, R: From { use timely::order::Product; - graph.scope().scoped::,_,_>("StronglyConnected", |scope| { + let outer = graph.scope(); + outer.scoped::,_,_>("StronglyConnected", |scope| { // Bring in edges and transposed edges. let edges = graph.enter(&scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); @@ -31,20 +32,21 @@ where let result = trim_edges(trim_edges(inner, edges), trans); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn trim_edges(cycle: VecCollection, edges: VecCollection) - -> VecCollection +fn trim_edges(cycle: VecCollection, edges: VecCollection) + -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, R: From { - edges.inner.scope().region_named("TrimEdges", |region| { + let outer = edges.inner.scope(); + outer.region_named("TrimEdges", |region| { let cycle = cycle.enter_region(region); let edges = edges.enter_region(region); @@ -62,6 +64,6 @@ where .join_core(labels, |e2,(e1,l1),l2| [((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))]) .filter(|(_,(l1,l2))| l1 == l2) .map(|((x1,x2),_)| (x2,x1)) - .leave_region() + .leave_region(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index 354ad32c2..8ba203fa4 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -2,16 +2,16 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; use crate::operators::*; use crate::hashable::Hashable; -fn _color(edges: VecCollection) -> VecCollection)> +fn _color(edges: VecCollection) -> VecCollection)> where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, { // need some bogus initial values. @@ -40,12 +40,12 @@ where /// a node "fires" once all of its neighbors with lower identifier have /// fired, and we apply `logic` to the new state of lower neighbors and /// the old state (input) of higher neighbors. -pub fn sequence( - state: VecCollection, - edges: VecCollection, - logic: F) -> VecCollection)> +pub fn sequence( + state: VecCollection, + edges: VecCollection, + logic: F) -> VecCollection)> where - G: Scope, + T: Timestamp + Lattice + Hash + Ord, 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..8c0678a53 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -1,6 +1,6 @@ //! Assign unique identifiers to records. -use timely::dataflow::Scope; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData, Hashable}; use crate::lattice::Lattice; @@ -8,7 +8,7 @@ use crate::operators::*; use crate::difference::Abelian; /// Assign unique identifiers to elements of a collection. -pub trait Identifiers { +pub trait Identifiers { /// Assign unique identifiers to elements of a collection. /// /// # Example @@ -27,16 +27,16 @@ pub trait Identifiers { /// .assert_empty(); /// }); /// ``` - fn identifiers(self) -> VecCollection; + fn identifiers(self) -> VecCollection; } -impl Identifiers for VecCollection +impl Identifiers for VecCollection where - G: Scope, + T: Timestamp + Lattice, D: ExchangeData + ::std::hash::Hash, R: ExchangeData + Abelian, { - fn identifiers(self) -> VecCollection { + fn identifiers(self) -> VecCollection { // The design here is that we iteratively develop a collection // of pairs (round, record), where each pair is a proposal that diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index f76f59d9c..f7401d892 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -1,13 +1,13 @@ //! Implementation of Parallel Prefix Sum -use timely::dataflow::Scope; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; use crate::operators::*; /// Extension trait for the prefix_sum method. -pub trait PrefixSum { +pub trait PrefixSum { /// Computes the prefix sum for each element in the collection. /// /// The prefix sum is data-parallel, in the sense that the sums are computed independently for @@ -16,12 +16,12 @@ pub trait PrefixSum { fn prefix_sum(self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; /// Determine the prefix sum at each element of `location`. - fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; } -impl PrefixSum for VecCollection +impl PrefixSum for VecCollection where - G: Scope, + T: Timestamp + Lattice, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, { @@ -29,7 +29,7 @@ where self.clone().prefix_sum_at(self.map(|(x,_)| x), zero, combine) } - fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { let combine1 = ::std::rc::Rc::new(combine); let combine2 = combine1.clone(); @@ -40,9 +40,9 @@ where } /// Accumulate data in `collection` into all powers-of-two intervals containing them. -pub fn aggregate(collection: VecCollection, combine: F) -> VecCollection +pub fn aggregate(collection: VecCollection, combine: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, @@ -73,13 +73,13 @@ where } /// Produces the accumulated values at each of the `usize` locations in `queries`. -pub fn broadcast( - ranges: VecCollection, - queries: VecCollection, +pub fn broadcast( + ranges: VecCollection, + queries: VecCollection, zero: D, - combine: F) -> VecCollection + combine: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + ::std::fmt::Debug, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, diff --git a/differential-dataflow/src/capture.rs b/differential-dataflow/src/capture.rs index 74cced0ca..65069bd74 100644 --- a/differential-dataflow/src/capture.rs +++ b/differential-dataflow/src/capture.rs @@ -230,7 +230,7 @@ pub mod source { use timely::dataflow::{Scope, Stream, operators::{Capability, CapabilitySet}}; use timely::dataflow::operators::generic::OutputBuilder; use timely::progress::Timestamp; - use timely::scheduling::SyncActivator; + use timely::scheduling::{Scheduler, SyncActivator}; // TODO(guswynn): implement this generally in timely struct DropActivator { @@ -250,12 +250,11 @@ pub mod source { /// The stream is built in the supplied `scope` and continues to run until /// the returned `Box` token is dropped. The `source_builder` argument /// is invoked with a `SyncActivator` that will re-activate the source. - pub fn build( - scope: G, + pub fn build( + scope: Scope, source_builder: B, - ) -> (Box, Stream>) + ) -> (Box, Stream>) where - G: Scope, B: FnOnce(SyncActivator) -> I, I: Iterator> + 'static, D: ExchangeData + Hash, @@ -560,9 +559,10 @@ pub mod sink { use timely::order::PartialOrder; use timely::progress::{Antichain, ChangeBatch, Timestamp}; - use timely::dataflow::{Scope, Stream}; + use timely::dataflow::Stream; use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::operators::generic::{builder_rc::OperatorBuilder, OutputBuilder}; + use timely::scheduling::Scheduler; use crate::{lattice::Lattice, ExchangeData}; use super::{Writer, Message, Progress}; @@ -573,13 +573,12 @@ pub mod sink { /// will *not* perform the consolidation on the stream's behalf. If this is not /// performed before calling the method, the recorded output may not be correctly /// reconstructed by readers. - pub fn build( - stream: Stream>, + pub fn build( + stream: Stream>, sink_hash: u64, updates_sink: Weak>, progress_sink: Weak>, ) where - G: Scope, BS: Writer> + 'static, D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, T: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a> + Timestamp + Lattice, @@ -742,9 +741,9 @@ pub mod sink { // use crate::lattice::Lattice; // /// Creates a Kafka source from supplied configuration information. -// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, Stream>) +// pub fn create_source(scope: T, addr: &str, topic: &str, group: &str) -> (Box, Stream>) // where -// G: Scope, +// T: Scope, // D: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, // T: ExchangeData + Hash + for<'a> serde::Deserialize<'a> + Timestamp + Lattice, // R: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, @@ -757,9 +756,9 @@ pub mod sink { // }) // } -// pub fn create_sink(stream: &Stream>, addr: &str, topic: &str) -> Box +// pub fn create_sink(stream: &Stream>, addr: &str, topic: &str) -> Box // where -// G: Scope, +// T: Scope, // D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, // T: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a> + Timestamp + Lattice, // R: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 6981cc3da..d33463863 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -10,7 +10,6 @@ use timely::Container; use timely::progress::Timestamp; -use timely::dataflow::scopes::Child; use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::*; @@ -22,7 +21,7 @@ use crate::difference::Abelian; /// in order to expose some of this functionality (e.g. negation, timestamp manipulation). Other actions /// on the containers, and streams of containers, are left to the container implementor to describe. #[derive(Clone)] -pub struct Collection { +pub struct Collection { /// The underlying timely dataflow stream. /// /// This field is exposed to support direct timely dataflow manipulation when required, but it is @@ -31,10 +30,10 @@ pub struct Collection { /// The timestamp in the data is required to always be at least the timestamp _of_ the data, in /// the timely-dataflow sense. If this invariant is not upheld, differential operators may behave /// unexpectedly. - pub inner: Stream, + pub inner: Stream, } -impl Collection { +impl Collection { /// Creates a new Collection from a timely dataflow stream. /// /// This method seems to be rarely used, with the `as_collection` method on streams being a more @@ -44,9 +43,9 @@ impl Collection { /// /// This stream should satisfy the timestamp invariant as documented on [Collection]; this /// method does not check it. - pub fn new(stream: Stream) -> Self { Self { inner: stream } } + pub fn new(stream: Stream) -> Self { Self { inner: stream } } } -impl Collection { +impl Collection { /// Creates a new collection accumulating the contents of the two collections. /// /// Despite the name, differential dataflow collections are unordered. This method is so named because the @@ -109,7 +108,7 @@ impl Collection { /// /// This method is a specialization of `enter` to the case where the nested scope is a region. /// It removes the need for an operator that adjusts the timestamp. - pub fn enter_region<'a>(self, child: &Child<'a, G, ::Timestamp>) -> Collection::Timestamp>, C> { + pub fn enter_region(self, child: &Scope) -> Self { self.inner .enter(child) .as_collection() @@ -134,7 +133,7 @@ impl Collection { /// ``` pub fn inspect_container(self, func: F) -> Self where - F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static, + F: FnMut(Result<(&T, &C), &[T]>)+'static, { self.inner .inspect_container(func) @@ -144,7 +143,7 @@ impl Collection { /// /// This probe is used to determine when the state of the Collection has stabilized and can /// be read out. - pub fn probe(self) -> (probe::Handle, Self) { + pub fn probe(self) -> (probe::Handle, Self) { let (handle, stream) = self.inner.probe(); (handle, stream.as_collection()) } @@ -154,11 +153,11 @@ impl Collection { /// In addition, a probe is also often use to limit the number of rounds of input in flight at any moment; a /// computation can wait until the probe has caught up to the input before introducing more rounds of data, to /// avoid swamping the system. - pub fn probe_with(self, handle: &probe::Handle) -> Self { + pub fn probe_with(self, handle: &probe::Handle) -> Self { Self::new(self.inner.probe_with(handle)) } /// The scope containing the underlying timely dataflow stream. - pub fn scope(&self) -> G { + pub fn scope(&self) -> Scope { self.inner.scope() } @@ -209,16 +208,16 @@ impl Collection { /// let result = scope.region(|child| { /// data.clone() /// .enter(child) - /// .leave() + /// .leave(&scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter<'a, T>(self, child: &Child<'a, G, T>) -> Collection, ::Timestamp, T>>::InnerContainer> + pub fn enter(self, child: &Scope) -> Collection>::InnerContainer> where - C: containers::Enter<::Timestamp, T, InnerContainer: Container>, - T: Refines<::Timestamp>, + C: containers::Enter, + TInner: Refines, { use timely::dataflow::channels::pact::Pipeline; self.inner @@ -249,9 +248,9 @@ impl Collection { /// data.results_in(summary1); /// }); /// ``` - pub fn results_in(self, step: ::Summary) -> Self + pub fn results_in(self, step: T::Summary) -> Self where - C: containers::ResultsIn<::Summary>, + C: containers::ResultsIn, { use timely::dataflow::channels::pact::Pipeline; self.inner @@ -262,14 +261,10 @@ impl Collection { } } -use timely::dataflow::scopes::ScopeParent; use timely::progress::timestamp::Refines; /// Methods requiring a nested scope. -impl<'a, G: Scope, T: Timestamp, C: Container> Collection, C> -where - C: containers::Leave, - T: Refines<::Timestamp>, +impl Collection { /// Returns the final value of a Collection from a nested scope to its containing scope. /// @@ -286,33 +281,37 @@ where /// let result = scope.region(|child| { /// data.clone() /// .enter(child) - /// .leave() + /// .leave(scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn leave(self) -> Collection>::OuterContainer> { + pub fn leave(self, outer: &Scope) -> Collection>::OuterContainer> + where + TOuter: Timestamp, + T: Refines, + C: containers::Leave, + { use timely::dataflow::channels::pact::Pipeline; self.inner - .leave() + .leave(outer) .unary(Pipeline, "Leave", move |_,_| move |input, output| { input.for_each(|time, data| output.session(&time).give_container(&mut std::mem::take(data).leave())); }) .as_collection() } -} -/// Methods requiring a region as the scope. -impl Collection, C> -{ /// Returns the value of a Collection from a nested region to its containing scope. /// /// This method is a specialization of `leave` to the case that of a nested region. /// It removes the need for an operator that adjusts the timestamp. - pub fn leave_region(self) -> Collection { + pub fn leave_region(self, outer: &Scope) -> Self + where + C: Clone + 'static, + { self.inner - .leave() + .leave(outer) .as_collection() } } @@ -325,8 +324,7 @@ pub mod vec { use timely::progress::Timestamp; use timely::order::Product; - use timely::dataflow::scopes::child::Iterative; - use timely::dataflow::{Scope, ScopeParent}; + use timely::dataflow::scope::Iterative; use timely::dataflow::operators::*; use timely::dataflow::operators::vec::*; @@ -344,18 +342,19 @@ pub mod vec { /// propagate changes through your functional computation and report the corresponding changes to the /// output collections. /// - /// Each vec collection has three generic parameters. The parameter `G` is for the scope in which the - /// collection exists; as you write more complicated programs you may wish to introduce nested scopes - /// (e.g. for iteration) and this parameter tracks the scope (for timely dataflow's benefit). The `D` - /// parameter is the type of data in your collection, for example `String`, or `(u32, Vec>)`. + /// Each vec collection has three generic parameters. The parameter `T` is the timestamp type of the + /// scope in which the collection exists; as you write more complicated programs you may wish to + /// introduce nested scopes (e.g. for iteration), and this parameter tracks the scope's timestamp + /// (for timely dataflow's benefit). The `D` parameter is the type of data in your collection, for + /// example `String`, or `(u32, Vec>)`. /// The `R` parameter represents the types of changes that the data undergo, and is most commonly (and /// defaults to) `isize`, representing changes to the occurrence count of each record. /// - /// This type definition instantiates the [`Collection`] type with a `Vec<(D, G::Timestamp, R)>`. - pub type Collection = super::Collection::Timestamp, R)>>; + /// This type definition instantiates the [`Collection`] type with a `Vec<(D, T, R)>`. + pub type Collection = super::Collection>; - impl Collection { + impl Collection { /// Creates a new collection by applying the supplied function to each input element. /// /// # Examples @@ -370,7 +369,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map(self, mut logic: L) -> Collection + pub fn map(self, mut logic: L) -> Collection where D2: Clone+'static, L: FnMut(D) -> D2 + 'static, @@ -397,7 +396,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map_in_place(self, mut logic: L) -> Collection + pub fn map_in_place(self, mut logic: L) -> Collection where L: FnMut(&mut D) + 'static, { @@ -421,9 +420,9 @@ pub mod vec { /// .flat_map(|x| 0 .. x); /// }); /// ``` - pub fn flat_map(self, mut logic: L) -> Collection + pub fn flat_map(self, mut logic: L) -> Collection where - G::Timestamp: Clone, + T: Clone, I: IntoIterator, L: FnMut(D) -> I + 'static, { @@ -445,7 +444,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn filter(self, mut logic: L) -> Collection + pub fn filter(self, mut logic: L) -> Collection where L: FnMut(&D) -> bool + 'static, { @@ -473,7 +472,7 @@ pub mod vec { /// x1.assert_eq(x2); /// }); /// ``` - pub fn explode(self, mut logic: L) -> Collection>::Output> + pub fn explode(self, mut logic: L) -> Collection>::Output> where D2: Clone+'static, R2: Semigroup+Multiply, @@ -507,12 +506,12 @@ pub mod vec { /// ); /// }); /// ``` - pub fn join_function(self, mut logic: L) -> Collection>::Output> + pub fn join_function(self, mut logic: L) -> Collection>::Output> where - G::Timestamp: Lattice, + T: Lattice, D2: Clone+'static, R2: Semigroup+Multiply, - I: IntoIterator, + I: IntoIterator, L: FnMut(D)->I+'static, { self.inner @@ -537,17 +536,16 @@ pub mod vec { /// let result = scope.iterative::(|child| { /// data.clone() /// .enter_at(child, |x| *x) - /// .leave() + /// .leave(&scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter_at<'a, T, F>(self, child: &Iterative<'a, G, T>, mut initial: F) -> Collection, D, R> + pub fn enter_at(self, child: &Iterative, mut initial: F) -> Collection, D, R> where - T: Timestamp+Hash, - F: FnMut(&D) -> T + Clone + 'static, - G::Timestamp: Hash, + TInner: Timestamp+Hash, + F: FnMut(&D) -> TInner + Clone + 'static, { self.inner .enter(child) @@ -565,10 +563,10 @@ pub mod vec { /// ordered, they should have the same order or compare equal once `func` is applied to them (this /// is because we advance the timely capability with the same logic, and it must remain `less_equal` /// to all of the data timestamps). - pub fn delay(self, func: F) -> Collection + pub fn delay(self, func: F) -> Collection where - G::Timestamp: Hash, - F: FnMut(&G::Timestamp) -> G::Timestamp + Clone + 'static, + T: Hash, + F: FnMut(&T) -> T + Clone + 'static, { let mut func1 = func.clone(); let mut func2 = func.clone(); @@ -602,9 +600,9 @@ pub mod vec { /// .inspect(|x| println!("error: {:?}", x)); /// }); /// ``` - pub fn inspect(self, func: F) -> Collection + pub fn inspect(self, func: F) -> Collection where - F: FnMut(&(D, G::Timestamp, R))+'static, + F: FnMut(&(D, T, R))+'static, { self.inner .inspect(func) @@ -628,9 +626,9 @@ pub mod vec { /// .inspect_batch(|t,xs| println!("errors @ {:?}: {:?}", t, xs)); /// }); /// ``` - pub fn inspect_batch(self, mut func: F) -> Collection + pub fn inspect_batch(self, mut func: F) -> Collection where - F: FnMut(&G::Timestamp, &[(D, G::Timestamp, R)])+'static, + F: FnMut(&T, &[(D, T, R)])+'static, { self.inner .inspect_batch(move |time, data| func(time, data)) @@ -660,7 +658,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable + Semigroup, - G::Timestamp: Lattice+Ord, + T: Lattice+Ord, { self.consolidate() .inspect(|x| panic!("Assertion failed: non-empty collection: {:?}", x)); @@ -668,7 +666,7 @@ pub mod vec { } /// Methods requiring an Abelian difference, to support negation. - impl, D: Clone+'static, R: Abelian+'static> Collection { + impl Collection { /// Assert if the collections are ever different. /// /// Because this is a dataflow fragment, the test is only applied as the computation is run. If the computation @@ -696,7 +694,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable, - G::Timestamp: Lattice+Ord, + T: Lattice+Ord, { self.negate() .concat(other) @@ -707,9 +705,9 @@ pub mod vec { use crate::trace::{Trace, Builder}; use crate::operators::arrange::{Arranged, TraceAgent}; - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -741,13 +739,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}; @@ -784,9 +782,9 @@ pub mod vec { /// .trace; /// }); /// ``` - pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> where - T2: for<'a> Trace= &'a K, ValOwn = V, Time=G::Timestamp, Diff: Abelian>+'static, + T2: for<'a> Trace= &'a K, ValOwn = V, Time=T, Diff: Abelian>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V, T2::Diff)>)+'static, { @@ -802,10 +800,10 @@ pub mod vec { /// Unlike `reduce_arranged`, this method may be called with an empty `input`, /// and it may not be safe to index into the first element. /// At least one of the two collections will be non-empty. - pub fn reduce_core(self, name: &str, logic: L) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where V: Clone+'static, - T2: for<'a> Trace=&'a K, ValOwn = V, Time=G::Timestamp>+'static, + T2: for<'a> Trace=&'a K, ValOwn = V, Time=T>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, { @@ -818,9 +816,9 @@ pub mod vec { } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, R1: crate::ExchangeData+Semigroup { @@ -839,7 +837,7 @@ pub mod vec { /// .distinct(); /// }); /// ``` - pub fn distinct(self) -> Collection { + pub fn distinct(self) -> Collection { self.distinct_core() } @@ -848,7 +846,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)) } @@ -870,16 +868,16 @@ 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)) - .reduce_abelian::<_,KeyBuilder,KeySpine,_>( + .reduce_abelian::<_,KeyBuilder,KeySpine,_>( name, move |k,s,t| t.push(((), thresh(k, &s[0].1))), |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, @@ -889,9 +887,9 @@ pub mod vec { } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup { @@ -910,17 +908,17 @@ pub mod vec { /// .count(); /// }); /// ``` - pub fn count(self) -> Collection { self.count_core() } + pub fn count(self) -> Collection { self.count_core() } /// Count for general integer differences. /// /// This method allows `count` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - pub fn count_core + 'static>(self) -> Collection { + pub fn count_core + 'static>(self) -> Collection { use crate::trace::implementations::{ValBuilder, ValSpine}; self.arrange_by_self_named("Arrange: Count") - .reduce_abelian::<_,ValBuilder,ValSpine,_>( + .reduce_abelian::<_,ValBuilder,ValSpine,_>( "Count", |_k,s,t| t.push((s[0].1.clone(), R2::from(1i8))), |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, @@ -930,9 +928,9 @@ pub mod vec { } /// Methods which require data be arrangeable. - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Clone + 'static + Lattice, D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup, { @@ -966,8 +964,8 @@ pub mod vec { /// and provide the function `reify` to produce owned keys and values.. pub fn consolidate_named(self, name: &str, reify: F) -> Self where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Tr: for<'a> crate::trace::Trace+'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Tr: for<'a> crate::trace::Trace+'static, Bu: crate::trace::Builder, F: Fn(Tr::Key<'_>, Tr::Val<'_>) -> D + 'static, { @@ -1025,82 +1023,82 @@ pub mod vec { use crate::trace::implementations::{KeySpine, KeyBatcher, KeyBuilder}; use crate::operators::arrange::Arrange; - impl Arrange> for Collection + impl Arrange> for Collection where - G: Scope, + T: Timestamp + Lattice, K: crate::ExchangeData + Hashable, V: crate::ExchangeData, R: crate::ExchangeData + Semigroup, { - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Bu: crate::trace::Builder, - Tr: crate::trace::Trace + 'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, + Tr: crate::trace::Trace + 'static, { - let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_, _, Ba, Bu, _>(self.inner, exchange, name) + let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),T,R)| (update.0).0.hashed().into()); + crate::operators::arrange::arrangement::arrange_core::<_, Ba, Bu, _>(self.inner, exchange, name) } } - impl Arrange> for Collection + impl Arrange> for Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Bu: crate::trace::Builder, - Tr: crate::trace::Trace + 'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, + Tr: crate::trace::Trace + 'static, { - let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_,_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) + let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),T,R)| (update.0).0.hashed().into()); + crate::operators::arrange::arrangement::arrange_core::<_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { /// Arranges a collection of `(Key, Val)` records by `Key`. /// /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// This trace is current for all times completed by the output stream, which can be used to /// safely identify the stable times and values in the trace. - pub fn arrange_by_key(self) -> Arranged>> { + pub fn arrange_by_key(self) -> Arranged>> { self.arrange_by_key_named("ArrangeByKey") } /// As `arrange_by_key` but with the ability to name the arrangement. - pub fn arrange_by_key_named(self, name: &str) -> Arranged>> { + pub fn arrange_by_key_named(self, name: &str) -> Arranged>> { self.arrange_named::,ValBuilder<_,_,_,_>,_>(name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { /// Arranges a collection of `Key` records by `Key`. /// /// This operator arranges a collection of records into a shared trace, whose contents it maintains. /// This trace is current for all times complete in the output stream, which can be used to safely /// identify the stable times and values in the trace. - pub fn arrange_by_self(self) -> Arranged>> { + pub fn arrange_by_self(self) -> Arranged>> { self.arrange_by_self_named("ArrangeBySelf") } /// As `arrange_by_self` but with the ability to name the arrangement. - pub fn arrange_by_self_named(self, name: &str) -> Arranged>> { + pub fn arrange_by_self_named(self, name: &str) -> Arranged>> { self.map(|k| (k, ())) .arrange_named::,KeyBuilder<_,_,_>,_>(name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -1124,7 +1122,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join(self, other: Collection) -> Collection>::Output> + pub fn join(self, other: Collection) -> Collection>::Output> where K: crate::ExchangeData, V2: crate::ExchangeData, @@ -1151,7 +1149,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join_map(self, other: Collection, mut logic: L) -> Collection>::Output> + pub fn join_map(self, other: Collection, mut logic: L) -> Collection>::Output> where R: Multiply, L: FnMut(&K, &V, &V2)->D+'static { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_key(); @@ -1179,7 +1177,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn semijoin(self, other: Collection) -> Collection>::Output> + pub fn semijoin(self, other: Collection) -> Collection>::Output> where R: Multiply { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_self(); @@ -1211,7 +1209,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn antijoin(self, other: Collection) -> Collection + pub fn antijoin(self, other: Collection) -> Collection where R: Multiply, R: Abelian+'static { self.clone().concat(self.semijoin(other).negate()) } @@ -1222,7 +1220,7 @@ pub mod vec { /// which produces something implementing `IntoIterator`, where the output collection will have an entry for /// every value returned by the iterator. /// - /// This trait is implemented for arrangements (`Arranged`) rather than collections. The `Join` trait + /// This trait is implemented for arrangements (`Arranged`) rather than collections. The `Join` trait /// contains the implementations for collections. /// /// # Examples @@ -1244,9 +1242,9 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join_core (self, stream2: Arranged, result: L) -> Collection>::Output> + pub fn join_core (self, stream2: Arranged, result: L) -> Collection>::Output> where - Tr2: for<'a> crate::trace::TraceReader=&'a K, Time=G::Timestamp>+Clone+'static, + Tr2: for<'a> crate::trace::TraceReader=&'a K, Time=T>+Clone+'static, R: Multiply, I: IntoIterator, L: FnMut(&K,&V,Tr2::Val<'_>)->I+'static, @@ -1258,18 +1256,18 @@ pub mod vec { } /// Conversion to a differential dataflow Collection. -pub trait AsCollection { +pub trait AsCollection { /// Converts the type to a differential dataflow collection. - fn as_collection(self) -> Collection; + fn as_collection(self) -> Collection; } -impl AsCollection for Stream { +impl AsCollection for Stream { /// Converts the type to a differential dataflow collection. /// /// By calling this method, you guarantee that the timestamp invariant (as documented on /// [Collection]) is upheld. This method will not check it. - fn as_collection(self) -> Collection { - Collection::::new(self) + fn as_collection(self) -> Collection { + Collection::::new(self) } } @@ -1294,11 +1292,11 @@ impl AsCollection for Stream { /// .assert_eq(data); /// }); /// ``` -pub fn concatenate(scope: &mut G, iterator: I) -> Collection +pub fn concatenate(scope: &mut Scope, iterator: I) -> Collection where - G: Scope, + T: Timestamp, C: Container, - I: IntoIterator>, + I: IntoIterator>, { scope .concatenate(iterator.into_iter().map(|x| x.inner)) diff --git a/differential-dataflow/src/dynamic/mod.rs b/differential-dataflow/src/dynamic/mod.rs index 19c5472a6..9efe58f6c 100644 --- a/differential-dataflow/src/dynamic/mod.rs +++ b/differential-dataflow/src/dynamic/mod.rs @@ -13,7 +13,6 @@ pub mod pointstamp; -use timely::dataflow::Scope; use timely::order::Product; use timely::progress::Timestamp; use timely::dataflow::operators::generic::{OutputBuilder, builder_rc::OperatorBuilder}; @@ -26,9 +25,8 @@ use crate::collection::AsCollection; use crate::dynamic::pointstamp::PointStamp; use crate::dynamic::pointstamp::PointStampSummary; -impl VecCollection +impl VecCollection>, D, R> where - G: Scope>>, D: Data, R: Semigroup+'static, T: Timestamp+Default, diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index 43a4987e8..0ef606ac7 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -9,7 +9,7 @@ use timely::progress::Timestamp; use timely::dataflow::operators::vec::Input as TimelyInput; use timely::dataflow::operators::vec::input::Handle; -use timely::dataflow::scopes::ScopeParent; +use timely::dataflow::Scope; use crate::Data; use crate::difference::Semigroup; @@ -41,7 +41,7 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection(&mut self) -> (InputSession<::Timestamp, D, R>, VecCollection) + fn new_collection(&mut self) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static; /// Create a new collection and input handle from initial data. /// @@ -67,7 +67,7 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection_from(&mut self, data: I) -> (InputSession<::Timestamp, I::Item, isize>, VecCollection) + fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection) where I: IntoIterator + 'static; /// Create a new collection and input handle from initial data. /// @@ -93,28 +93,28 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection_from_raw(&mut self, data: I) -> (InputSession<::Timestamp, D, R>, VecCollection) - where I: IntoIterator::Timestamp,R)>+'static, D: Data, R: Semigroup+'static; + fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection) + where I: IntoIterator+'static, D: Data, R: Semigroup+'static; } use crate::lattice::Lattice; -impl Input for G where ::Timestamp: Lattice { - fn new_collection(&mut self) -> (InputSession<::Timestamp, D, R>, VecCollection) +impl Input for Scope { + fn new_collection(&mut self) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static, { let (handle, stream) = self.new_input(); (InputSession::from(handle), stream.as_collection()) } - fn new_collection_from(&mut self, data: I) -> (InputSession<::Timestamp, I::Item, isize>, VecCollection) + fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection) where I: IntoIterator+'static, I::Item: Data { - self.new_collection_from_raw(data.into_iter().map(|d| (d, ::minimum(), 1))) + self.new_collection_from_raw(data.into_iter().map(|d| (d, ::minimum(), 1))) } - fn new_collection_from_raw(&mut self, data: I) -> (InputSession<::Timestamp, D, R>, VecCollection) + fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static, - I: IntoIterator::Timestamp,R)>+'static, + I: IntoIterator+'static, { use timely::dataflow::operators::ToStream; @@ -122,7 +122,8 @@ impl Input for G where ::Timestamp: Lattice { let source = data.to_stream(self).as_collection(); (InputSession::from(handle), stream.as_collection().concat(source)) - }} + } +} /// An input session wrapping a single timely dataflow capability. /// @@ -198,9 +199,9 @@ impl InputSession { impl InputSession { /// Introduces a handle as collection. - pub fn to_collection(&mut self, scope: &mut G) -> VecCollection + pub fn to_collection(&mut self, scope: &mut Scope) -> VecCollection where - G: ScopeParent, + T: timely::order::TotalOrder, { scope .input_from(&mut self.handle) diff --git a/differential-dataflow/src/logging.rs b/differential-dataflow/src/logging.rs index 4fb6ef430..1ba9833f8 100644 --- a/differential-dataflow/src/logging.rs +++ b/differential-dataflow/src/logging.rs @@ -10,9 +10,8 @@ pub type DifferentialEventBuilder = timely::container::CapacityContainerBuilder< pub type Logger = ::timely::logging_core::TypedLogger; /// Enables logging of differential dataflow events. -pub fn enable(worker: &mut timely::worker::Worker, writer: W) -> Option> +pub fn enable(worker: &mut timely::worker::Worker, writer: W) -> Option> where - A: timely::communication::Allocate, W: std::io::Write + 'static, { worker.log_register().and_then(|mut log_register| { diff --git a/differential-dataflow/src/operators/arrange/agent.rs b/differential-dataflow/src/operators/arrange/agent.rs index 715542f55..7ae253d2c 100644 --- a/differential-dataflow/src/operators/arrange/agent.rs +++ b/differential-dataflow/src/operators/arrange/agent.rs @@ -9,6 +9,7 @@ use timely::dataflow::operators::generic::{OperatorInfo, source}; use timely::progress::Timestamp; use timely::progress::{Antichain, frontier::AntichainRef}; use timely::dataflow::operators::CapabilitySet; +use timely::scheduling::Scheduler; use crate::trace::{Trace, TraceReader, BatchReader}; use crate::trace::wrappers::rc::TraceBox; @@ -101,7 +102,7 @@ impl TraceAgent { }; let writer = TraceWriter::new( - vec![::minimum()], + vec![Tr::Time::minimum()], Rc::downgrade(&trace), queues, ); @@ -125,7 +126,7 @@ impl TraceAgent { .borrow_mut() .trace .map_batches(|batch| { - new_queue.push_back(TraceReplayInstruction::Batch(batch.clone(), Some(::minimum()))); + new_queue.push_back(TraceReplayInstruction::Batch(batch.clone(), Some(Tr::Time::minimum()))); upper = Some(batch.upper().clone()); }); @@ -215,17 +216,13 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import(&mut self, scope: &G) -> Arranged> - where - G: Scope, + pub fn import(&mut self, scope: &Scope) -> Arranged> { self.import_named(scope, "ArrangedSource") } /// Same as `import`, but allows to name the source. - pub fn import_named(&mut self, scope: &G, name: &str) -> Arranged> - where - G: Scope, + pub fn import_named(&mut self, scope: &Scope, name: &str) -> Arranged> { // Drop ShutdownButton and return only the arrangement. self.import_core(scope, name).0 @@ -278,9 +275,7 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_core(&mut self, scope: &G, name: &str) -> (Arranged>, ShutdownButton>) - where - G: Scope, + pub fn import_core(&mut self, scope: &Scope, name: &str) -> (Arranged>, ShutdownButton>) { let trace = self.clone(); @@ -393,9 +388,8 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_frontier(&mut self, scope: &G, name: &str) -> (Arranged>>, ShutdownButton>) + pub fn import_frontier(&mut self, scope: &Scope, name: &str) -> (Arranged>>, ShutdownButton>) where - G: Scope, Tr: TraceReader, { // This frontier describes our only guarantee on the compaction frontier. @@ -411,9 +405,8 @@ impl TraceAgent { /// /// Invoking this method with an `until` of `Antichain::new()` will perform no filtering, as the empty /// frontier indicates the end of times. - pub fn import_frontier_core(&mut self, scope: &G, name: &str, since: Antichain, until: Antichain) -> (Arranged>>, ShutdownButton>) + pub fn import_frontier_core(&mut self, scope: &Scope, name: &str, since: Antichain, until: Antichain) -> (Arranged>>, ShutdownButton>) where - G: Scope, Tr: TraceReader, { let trace = self.clone(); diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index cd1bd0894..4034bb9a1 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -25,6 +25,8 @@ use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; use timely::progress::Antichain; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::{Data, VecCollection, AsCollection}; use crate::difference::Semigroup; @@ -41,9 +43,8 @@ use super::TraceAgent; /// /// An `Arranged` allows multiple differential operators to share the resources (communication, /// computation, memory) required to produce and maintain an indexed representation of a collection. -pub struct Arranged +pub struct Arranged where - G: Scope, Tr: TraceReader+Clone, { /// A stream containing arranged updates. @@ -51,16 +52,15 @@ where /// This stream contains the same batches of updates the trace itself accepts, so there should /// be no additional overhead to receiving these records. The batches can be navigated just as /// the batches in the trace, by key and by value. - pub stream: Stream>, + pub stream: Stream>, /// A shared trace, updated by the `Arrange` operator and readable by others. pub trace: Tr, // TODO : We might have an `Option>` here, which `as_collection` sets and // returns when invoked, so as to not duplicate work with multiple calls to `as_collection`. } -impl Clone for Arranged +impl Clone for Arranged where - G: Scope, Tr: TraceReader + Clone, { fn clone(&self) -> Self { @@ -71,13 +71,11 @@ where } } -use ::timely::dataflow::scopes::Child; use ::timely::progress::timestamp::Refines; use timely::Container; -impl Arranged +impl Arranged where - G: Scope, Tr: TraceReader + Clone, { /// Brings an arranged collection into a nested scope. @@ -85,10 +83,10 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter<'a, TInner>(self, child: &Child<'a, G, TInner>) - -> Arranged, TraceEnter> + pub fn enter(self, child: &Scope) + -> Arranged> where - TInner: Refines+Lattice+Timestamp+Clone, + TInner: Refines+Lattice, { Arranged { stream: self.stream.enter(child).map(|bw| BatchEnter::make_from(bw)), @@ -100,8 +98,7 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn enter_region<'a>(self, child: &Child<'a, G, G::Timestamp>) - -> Arranged, Tr> { + pub fn enter_region(self, child: &Scope) -> Self { Arranged { stream: self.stream.enter(child), trace: self.trace, @@ -113,10 +110,10 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter_at<'a, TInner, F, P>(self, child: &Child<'a, G, TInner>, logic: F, prior: P) - -> Arranged, TraceEnterAt> + pub fn enter_at(self, child: &Scope, logic: F, prior: P) + -> Arranged> where - TInner: Refines+Lattice+Timestamp+Clone+'static, + TInner: Refines+Lattice+'static, F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone+'static, P: FnMut(&TInner)->Tr::Time+Clone+'static, { @@ -132,7 +129,7 @@ where /// /// This method is like `self.stream.flat_map`, except that it produces containers /// directly, rather than form a container of containers as `flat_map` would. - pub fn as_container(self, mut logic: L) -> crate::Collection + pub fn as_container(self, mut logic: L) -> crate::Collection where I: IntoIterator, L: FnMut(Tr::Batch) -> I+'static, @@ -152,10 +149,10 @@ where /// Flattens the stream into a `VecCollection`. /// - /// The underlying `Stream>>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. - pub fn as_collection(self, mut logic: L) -> VecCollection + pub fn as_collection(self, mut logic: L) -> VecCollection where L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> D+'static, { @@ -164,14 +161,14 @@ where /// Flattens the stream into a `VecCollection`. /// - /// The underlying `Stream>>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. /// /// The method takes `K` and `V` as generic arguments, in order to constrain the reference types to support /// cloning into owned types. If this bound does not work, the `as_collection` method allows arbitrary logic /// on the reference types. - pub fn as_vecs(self) -> VecCollection + pub fn as_vecs(self) -> VecCollection where K: crate::ExchangeData, V: crate::ExchangeData, @@ -184,7 +181,7 @@ where /// /// The supplied logic may produce an iterator over output values, allowing either /// filtering or flat mapping as part of the extraction. - pub fn flat_map_ref(self, logic: L) -> VecCollection + pub fn flat_map_ref(self, logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -199,7 +196,7 @@ where /// /// This method exists for streams of batches without the corresponding arrangement. /// If you have the arrangement, its `flat_map_ref` method is equivalent to this. - pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection + pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -231,29 +228,28 @@ where use crate::difference::Multiply; // Direct join implementations. -impl Arranged +impl Arranged where - G: Scope, - T1: TraceReader + Clone + 'static, + Tr1: TraceReader + Clone + 'static, { /// A convenience method to join and produce `VecCollection` output. /// /// Avoid this method, as it is likely to evolve into one without the `VecCollection` opinion. - pub fn join_core(self, other: Arranged, mut result: L) -> VecCollection>::Output> + pub fn join_core(self, other: Arranged, mut result: L) -> VecCollection>::Output> where - T2: for<'a> TraceReader=T1::Key<'a>,Time=T1::Time>+Clone+'static, - T1::Diff: Multiply, + Tr2: for<'a> TraceReader=Tr1::Key<'a>,Time=Tr1::Time>+Clone+'static, + Tr1::Diff: Multiply, I: IntoIterator, - L: FnMut(T1::Key<'_>,T1::Val<'_>,T2::Val<'_>)->I+'static + L: FnMut(Tr1::Key<'_>,Tr1::Val<'_>,Tr2::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: Tr1::Key<'_>, v1: Tr1::Val<'_>, v2: Tr2::Val<'_>, t: &Tr1::Time, r1: &Tr1::Diff, r2: &Tr2::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())) }; use crate::operators::join::join_traces; - join_traces::<_, _, _, _, crate::consolidation::ConsolidatingContainerBuilder<_>>( + join_traces::<_, _, _, crate::consolidation::ConsolidatingContainerBuilder<_>>( self, other, move |k, v1, v2, t, d1, d2, c| { @@ -268,26 +264,24 @@ where // Direct reduce implementations. use crate::difference::Abelian; -impl Arranged +impl Arranged where - G: Scope, - T1: TraceReader + Clone + 'static, + Tr1: TraceReader + Clone + 'static, { /// A direct implementation of `ReduceCore::reduce_abelian`. - pub fn reduce_abelian(self, name: &str, mut logic: L, push: P) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L, push: P) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>= T1::Key<'a>, + Tr2: for<'a> Trace< + Key<'a>= Tr1::Key<'a>, ValOwn: Data, - Time=T1::Time, + Time=Tr1::Time, Diff: Abelian, >+'static, - Bu: Builder, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { - self.reduce_core::<_,Bu,T2,_>(name, move |key, input, output, change| { + self.reduce_core::<_,Bu,Tr2,_>(name, move |key, input, output, change| { if !input.is_empty() { logic(key, input, change); } @@ -297,63 +291,62 @@ where } /// A direct implementation of `ReduceCore::reduce_core`. - pub fn reduce_core(self, name: &str, logic: L, push: P) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L, push: P) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>=T1::Key<'a>, + Tr2: for<'a> Trace< + Key<'a>=Tr1::Key<'a>, ValOwn: Data, - Time=T1::Time, + Time=Tr1::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, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn, Tr2::Diff)>, &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { use crate::operators::reduce::reduce_trace; - reduce_trace::<_,_,Bu,_,_,_>(self, name, logic, push) + reduce_trace::<_,Bu,_,_,_>(self, name, logic, push) } } -impl<'a, G, Tr> Arranged, Tr> +impl Arranged where - G: Scope, Tr: TraceReader + Clone, { /// Brings an arranged collection out of a nested region. /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn leave_region(self) -> Arranged { + pub fn leave_region(self, outer: &Scope) -> Self + { use timely::dataflow::operators::Leave; Arranged { - stream: self.stream.leave(), + stream: self.stream.leave(outer), trace: self.trace, } } } /// A type that can be arranged as if a collection of updates. -pub trait Arrange : Sized +pub trait Arrange : Sized where - G: Scope, + T: Timestamp + Lattice, { /// Arranges updates into a shared trace. - fn arrange(self) -> Arranged> + fn arrange(self) -> Arranged> where - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace + 'static, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace + 'static, { self.arrange_named::("Arrange") } /// Arranges updates into a shared trace, with a supplied name. - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace + 'static, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace + 'static, ; } @@ -362,13 +355,12 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// It uses the supplied parallelization contract to distribute the data, which does not need to /// be consistently by key (though this is the most common). -pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> +pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> where - G: Scope, - P: ParallelizationContract, - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace+'static, + P: ParallelizationContract, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace+'static, { // The `Arrange` operator is tasked with reacting to an advancing input // frontier by producing the sequence of batches whose lower and upper @@ -400,7 +392,7 @@ where let mut batcher = Ba::new(logger.clone(), info.global_id); // Capabilities for the lower envelope of updates in `batcher`. - let mut capabilities = Antichain::>::new(); + let mut capabilities = Antichain::>::new(); let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); @@ -414,7 +406,7 @@ where *reader_ref = Some(reader_local); // Initialize to the minimal input frontier. - let mut prev_frontier = Antichain::from_elem(::minimum()); + let mut prev_frontier = Antichain::from_elem(Tr::Time::minimum()); move |(input, frontier), output| { diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index a8322eaa3..33b62a0e8 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -59,7 +59,7 @@ //! use differential_dataflow::operators::arrange::upsert; //! //! let stream = scope.input_from(&mut input); -//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine,String,String>(stream, &"test"); +//! let arranged = upsert::arrange_from_upsert::, ValSpine,String,String>(stream, &"test"); //! //! arranged //! .as_collection(|k,v| (k.clone(), v.clone())) @@ -101,12 +101,13 @@ use std::collections::{BinaryHeap, BTreeMap}; use timely::order::{PartialOrder, TotalOrder}; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::Exchange; -use timely::progress::Timestamp; -use timely::progress::Antichain; +use timely::progress::{Antichain, Timestamp}; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::operators::arrange::arrangement::Arranged; use crate::trace::{Builder, Description}; @@ -127,12 +128,11 @@ use super::TraceAgent; /// This method is only implemented for totally ordered times, as we do not yet /// understand what a "sequence" of upserts would mean for partially ordered /// timestamps. -pub fn arrange_from_upsert( - stream: Stream, G::Timestamp)>>, +pub fn arrange_from_upsert( + stream: Stream, Tr::Time)>>, name: &str, -) -> Arranged> +) -> Arranged> where - G: Scope, K: ExchangeData+Hashable+std::hash::Hash, V: ExchangeData, Tr: for<'a> Trace< @@ -141,7 +141,7 @@ where Time: TotalOrder+ExchangeData, Diff=isize, >+'static, - Bu: Builder, Output = Tr::Batch>, + Bu: Builder, Output = Tr::Batch>, { let mut reader: Option> = None; @@ -150,7 +150,7 @@ where let reader = &mut reader; - let exchange = Exchange::new(move |update: &(K,Option,G::Timestamp)| (update.0).hashed().into()); + let exchange = Exchange::new(move |update: &(K,Option,Tr::Time)| (update.0).hashed().into()); let scope = stream.scope(); stream.unary_frontier(exchange, name, move |_capability, info| { @@ -159,7 +159,7 @@ where let logger = scope.logger_for::("differential/arrange").map(Into::into); // Tracks the lower envelope of times in `priority_queue`. - let mut capabilities = Antichain::>::new(); + let mut capabilities = Antichain::>::new(); // Form the trace we will both use internally and publish. let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); @@ -173,10 +173,10 @@ where *reader = Some(reader_local.clone()); // Tracks the input frontier, used to populate the lower bound of new batches. - let mut prev_frontier = Antichain::from_elem(::minimum()); + let mut prev_frontier = Antichain::from_elem(Tr::Time::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| { @@ -279,7 +279,7 @@ where updates.sort(); builder.push(&mut updates); } - let description = Description::new(prev_frontier.clone(), upper.clone(), Antichain::from_elem(G::Timestamp::minimum())); + let description = Description::new(prev_frontier.clone(), upper.clone(), Antichain::from_elem(Tr::Time::minimum())); let batch = builder.done(description); prev_frontier.clone_from(&upper); diff --git a/differential-dataflow/src/operators/count.rs b/differential-dataflow/src/operators/count.rs index 713c73c86..8f4e7ad7a 100644 --- a/differential-dataflow/src/operators/count.rs +++ b/differential-dataflow/src/operators/count.rs @@ -1,7 +1,7 @@ //! Count the number of occurrences of each element. use timely::order::TotalOrder; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; @@ -14,7 +14,7 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `count` differential dataflow method. -pub trait CountTotal, K: ExchangeData, R: Semigroup> : Sized { +pub trait CountTotal : Sized { /// Counts the number of occurrences of each element. /// /// # Examples @@ -30,7 +30,7 @@ pub trait CountTotal, K: ExchangeDat /// .count_total(); /// }); /// ``` - fn count_total(self) -> VecCollection { + fn count_total(self) -> VecCollection { self.count_total_core() } @@ -39,39 +39,38 @@ pub trait CountTotal, K: ExchangeDat /// This method allows `count_total` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn count_total_core + 'static>(self) -> VecCollection; + fn count_total_core + 'static>(self) -> VecCollection; } -impl CountTotal for VecCollection +impl CountTotal for VecCollection where - G: Scope, + T: Timestamp + TotalOrder + Lattice + Ord, { - fn count_total_core + 'static>(self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { self.arrange_by_self_named("Arrange: CountTotal") .count_total_core() } } -impl CountTotal for Arranged +impl CountTotal for Arranged where - G: Scope, - T1: for<'a> TraceReader< + Tr: for<'a> TraceReader< Key<'a> = &'a K, Val<'a>=&'a (), Time: TotalOrder, - Diff: ExchangeData+Semigroup> + Diff: ExchangeData+Semigroup> >+Clone+'static, K: ExchangeData, { - fn count_total_core + 'static>(self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { let mut trace = self.trace.clone(); self.stream.unary_frontier(Pipeline, "CountTotal", move |_,_| { // tracks the lower and upper limit of received batches. - let mut lower_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); - let mut upper_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); + let mut lower_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); + let mut upper_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); move |(input, _frontier), output| { @@ -103,13 +102,13 @@ where let (mut trace_cursor, trace_storage) = trace.cursor_through(lower_limit.borrow()).unwrap(); while let Some(key) = batch_cursor.get_key(&batch_storage) { - let mut count: Option = None; + let mut count: Option = None; trace_cursor.seek_key(&trace_storage, key); 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(Tr::owned_diff(diff)); } }); } @@ -117,14 +116,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()), Tr::owned_time(time), R2::from(-1i8))); } } count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(Tr::owned_diff(diff)); } if let Some(count) = count.as_ref() { if !count.is_zero() { - session.give(((key.clone(), count.clone()), T1::owned_time(time), R2::from(1i8))); + session.give(((key.clone(), count.clone()), Tr::owned_time(time), R2::from(1i8))); } } }); diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index e5486bb6e..140e8888a 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -37,7 +37,7 @@ use timely::progress::Timestamp; use timely::order::Product; use timely::dataflow::*; -use timely::dataflow::scopes::child::Iterative; +use timely::dataflow::scope::Iterative; use timely::dataflow::operators::{Feedback, ConnectLoop}; use timely::dataflow::operators::feedback::Handle; @@ -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 { /// Iteratively apply `logic` to the source collection until convergence. /// /// Importantly, this method does not automatically consolidate results. @@ -73,17 +73,18 @@ pub trait Iterate, D: Data, R: Semigroup> { /// }); /// }); /// ``` - fn iterate(self, logic: F) -> VecCollection + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>; + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>; } -impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { - fn iterate(self, logic: F) -> VecCollection +impl Iterate for VecCollection { + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>, { - self.inner.scope().scoped("Iterate", |subgraph| { + let outer = self.inner.scope(); + outer.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // // this could be much more succinct if we returned the collection @@ -93,16 +94,17 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat let (variable, collection) = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); let result = logic(subgraph.clone(), collection); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } } -impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { - fn iterate(mut self, logic: F) -> VecCollection +impl Iterate for Scope { + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>, { + let outer = self.clone(); self.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // @@ -113,7 +115,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter let (variable, collection) = Variable::new(subgraph, Product::new(Default::default(), 1)); let result = logic(subgraph.clone(), collection); variable.set(result.clone()); - result.leave() + result.leave(&outer) } ) } @@ -146,7 +148,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// let result = collection.map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate(); /// variable.set(result.clone()); -/// result.leave() +/// result.leave(&scope) /// }); /// }) /// ``` @@ -187,23 +189,23 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// By iteratively developing a variable of the *edits* to the input, we can produce and circulate /// a smaller volume of updates. This can be especially impactful when the initial collection is /// large, and the edits to perform are relatively smaller. -pub struct Variable +pub struct Variable where - G: Scope, + T: Timestamp + Lattice, C: Container, { - feedback: Handle, - source: Option>, - step: ::Summary, + feedback: Handle, + source: Option>, + step: T::Summary, } /// A `Variable` specialized to a vector container of update triples (data, time, diff). -pub type VecVariable = Variable::Timestamp, R)>>; +pub type VecVariable = Variable>; -impl Variable +impl Variable where - G: Scope, - C: crate::collection::containers::ResultsIn<::Summary>, + T: Timestamp + Lattice, + C: crate::collection::containers::ResultsIn, { /// Creates a new initially empty `Variable` and its associated `Collection`. /// @@ -216,9 +218,9 @@ where /// will produce its fixed point in the outer scope. /// /// In a non-iterative scope the mechanics are the same, but the interpretation varies. - pub fn new(scope: &mut G, step: ::Summary) -> (Self, Collection) { + pub fn new(scope: &mut Scope, step: T::Summary) -> (Self, Collection) { let (feedback, updates) = scope.feedback(step.clone()); - let collection = Collection::::new(updates); + let collection = Collection::::new(updates); (Self { feedback, source: None, step }, collection) } @@ -247,9 +249,9 @@ where /// adding the source, doing the logic, then subtracting the source, it is appropriate to do. /// For example, if the logic modifies a few records it is possible to produce this update /// directly without using the backstop implementation this method provides. - pub fn new_from(source: Collection, step: ::Summary) -> (Self, Collection) where C: Clone + crate::collection::containers::Negate { + pub fn new_from(source: Collection, step: T::Summary) -> (Self, Collection) where C: Clone + crate::collection::containers::Negate { let (feedback, updates) = source.inner.scope().feedback(step.clone()); - let collection = Collection::::new(updates).concat(source.clone()); + let collection = Collection::::new(updates).concat(source.clone()); (Variable { feedback, source: Some(source.negate()), step }, collection) } @@ -257,7 +259,7 @@ where /// /// This method binds the `Variable` to be equal to the supplied collection, /// which may be recursively defined in terms of the variable itself. - pub fn set(mut self, mut result: Collection) { + pub fn set(mut self, mut result: Collection) { if let Some(source) = self.source.take() { result = result.concat(source); } diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index dae4d31b3..7d514cfec 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -9,10 +9,11 @@ use timely::{Accountable, ContainerBuilder}; use timely::container::PushInto; use timely::order::PartialOrder; use timely::progress::Timestamp; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::{Operator, OutputBuilderSession, Session}; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; use crate::lattice::Lattice; use crate::operators::arrange::Arranged; @@ -66,12 +67,11 @@ impl, D> PushInto for EffortBuilder { /// The "correctness" of this method depends heavily on the behavior of the supplied `result` function. /// /// [`AsCollection`]: crate::collection::AsCollection -pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> Stream +pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> Stream 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, + Tr1: TraceReader+Clone+'static, + Tr2: for<'a> TraceReader=Tr1::Key<'a>, Time = Tr1::Time>+Clone+'static, + L: FnMut(Tr1::Key<'_>,Tr1::Val<'_>,Tr2::Val<'_>,&Tr1::Time,&Tr1::Diff,&Tr2::Diff,&mut JoinSession>)+'static, CB: ContainerBuilder, { // Rename traces for symmetry from here on out. @@ -97,8 +97,8 @@ where // the physical compaction frontier of their corresponding trace. // Should we ever *drop* a trace, these are 1. much harder to maintain correctly, but 2. no longer used. use timely::progress::frontier::Antichain; - let mut acknowledged1 = Antichain::from_elem(::minimum()); - let mut acknowledged2 = Antichain::from_elem(::minimum()); + let mut acknowledged1 = Antichain::from_elem(Tr1::Time::minimum()); + let mut acknowledged2 = Antichain::from_elem(Tr1::Time::minimum()); // deferred work of batches from each input. let mut todo1 = std::collections::VecDeque::new(); diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 909199295..4fcaefd08 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -9,9 +9,10 @@ use crate::Data; use timely::progress::frontier::Antichain; use timely::progress::Timestamp; -use timely::dataflow::*; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; @@ -32,14 +33,13 @@ use crate::trace::TraceReader; /// the value updates, as appropriate for the container. It is critical that it clear the container as /// the operator has no ability to do this otherwise, and failing to do so represents a leak from one /// key's computation to another, and will likely introduce non-determinism. -pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L, mut push: P) -> Arranged> +pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L, mut push: P) -> Arranged> where - G: Scope, - T1: TraceReader + Clone + 'static, - T2: for<'a> Trace=T1::Key<'a>, 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, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Tr1: TraceReader + Clone + 'static, + Tr2: for<'a> Trace=Tr1::Key<'a>, ValOwn: Data, Time = Tr1::Time> + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn,Tr2::Diff)>, &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { let mut result_trace = None; @@ -54,7 +54,7 @@ where let logger = scope.logger_for::("differential/arrange").map(Into::into); let activator = Some(scope.activator_for(operator_info.address.clone())); - let mut empty = T2::new(operator_info.clone(), logger.clone(), activator); + let mut empty = Tr2::new(operator_info.clone(), logger.clone(), activator); // If there is default exert logic set, install it. if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { empty.set_exert_logic(exert_logic); @@ -66,26 +66,26 @@ where *result_trace = Some(output_reader.clone()); - let mut new_interesting_times = Vec::::new(); + let mut new_interesting_times = Vec::::new(); // Our implementation maintains a list of outstanding `(key, time)` synthetic interesting times, // sorted by (key, time), as well as capabilities for the lower envelope of the times. - let mut pending_keys = T1::KeyContainer::with_capacity(0); - let mut pending_time = T1::TimeContainer::with_capacity(0); - let mut next_pending_keys = T1::KeyContainer::with_capacity(0); - let mut next_pending_time = T1::TimeContainer::with_capacity(0); - let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); + let mut pending_keys = Tr1::KeyContainer::with_capacity(0); + let mut pending_time = Tr1::TimeContainer::with_capacity(0); + let mut next_pending_keys = Tr1::KeyContainer::with_capacity(0); + let mut next_pending_time = Tr1::TimeContainer::with_capacity(0); + let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); // buffers and logic for computing per-key interesting times "efficiently". - let mut interesting_times = Vec::::new(); + let mut interesting_times = Vec::::new(); // Upper and lower frontiers for the pending input and output batches to process. - let mut upper_limit = Antichain::from_elem(::minimum()); - let mut lower_limit = Antichain::from_elem(::minimum()); + let mut upper_limit = Antichain::from_elem(::minimum()); + let mut lower_limit = Antichain::from_elem(::minimum()); // Output batches may need to be built piecemeal, and these temp storage help there. - let mut output_upper = Antichain::from_elem(::minimum()); - let mut output_lower = Antichain::from_elem(::minimum()); + let mut output_upper = Antichain::from_elem(::minimum()); + let mut output_lower = Antichain::from_elem(::minimum()); move |(input, frontier), output| { @@ -130,14 +130,14 @@ where if capabilities.iter().any(|c| !upper_limit.less_equal(c.time())) { // cursors for navigating input and output traces. - let (mut source_cursor, ref source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); - let (mut output_cursor, ref output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); + let (mut source_cursor, ref source_storage): (Tr1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); + let (mut output_cursor, ref output_storage): (Tr2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); let (mut batch_cursor, ref batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); // Prepare an output buffer and builder for each capability. // 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::<(Tr1::Time, Vec<(Tr2::ValOwn, Tr1::Time, Tr2::Diff)>)>::new(); let mut builders = Vec::new(); for cap in capabilities.iter() { buffers.push((cap.time().clone(), Vec::new())); @@ -171,7 +171,7 @@ where let prior_pos = pending_pos; interesting_times.clear(); while pending_keys.get(pending_pos) == Some(key) { - let owned_time = T1::owned_time(pending_time.index(pending_pos)); + let owned_time = Tr1::owned_time(pending_time.index(pending_pos)); if !upper_limit.less_equal(&owned_time) { interesting_times.push(owned_time); } pending_pos += 1; } @@ -201,7 +201,7 @@ where // Merge novel pending times with any prior pending times we did not process. // TODO: This could be a merge, not a sort_dedup, because both lists should be sorted. for pos in prior_pos .. pending_pos { - let owned_time = T1::owned_time(pending_time.index(pos)); + let owned_time = Tr1::owned_time(pending_time.index(pos)); if upper_limit.less_equal(&owned_time) { new_interesting_times.push(owned_time); } } sort_dedup(&mut new_interesting_times); @@ -252,7 +252,7 @@ where if output_upper.borrow() != output_lower.borrow() { - let description = Description::new(output_lower.clone(), output_upper.clone(), Antichain::from_elem(G::Timestamp::minimum())); + let description = Description::new(output_lower.clone(), output_upper.clone(), Antichain::from_elem(Tr1::Time::minimum())); let batch = builder.done(description); // ship batch to the output, and commit to the output trace. @@ -272,10 +272,10 @@ where pending_time.clear(); std::mem::swap(&mut next_pending_time, &mut pending_time); // Update `capabilities` to reflect pending times. - let mut frontier = Antichain::::new(); - let mut owned_time = T1::Time::minimum(); + let mut frontier = Antichain::::new(); + let mut owned_time = Tr1::Time::minimum(); for pos in 0 .. pending_time.len() { - T1::clone_time_onto(pending_time.index(pos), &mut owned_time); + Tr1::clone_time_onto(pending_time.index(pos), &mut owned_time); frontier.insert_ref(&owned_time); } capabilities.downgrade(frontier); diff --git a/differential-dataflow/src/operators/threshold.rs b/differential-dataflow/src/operators/threshold.rs index 91bbd16fe..22dfec32b 100644 --- a/differential-dataflow/src/operators/threshold.rs +++ b/differential-dataflow/src/operators/threshold.rs @@ -4,7 +4,7 @@ //! `distinct` and `distinct_u` operators for the case in which time is totally ordered. use timely::order::TotalOrder; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; @@ -17,9 +17,9 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `distinct` differential dataflow method. -pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> : Sized { +pub trait ThresholdTotal : Sized { /// Reduces the collection to one occurrence of each distinct element. - fn threshold_semigroup(self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -39,7 +39,7 @@ pub trait ThresholdTotal, K: Exchang /// .threshold_total(|_,c| c % 2); /// }); /// ``` - fn threshold_totalR2+'static>(self, mut thresh: F) -> VecCollection { + fn threshold_totalR2+'static>(self, mut thresh: F) -> VecCollection { self.threshold_semigroup(move |key, new, old| { let mut new = thresh(key, new); if let Some(old) = old { @@ -69,7 +69,7 @@ pub trait ThresholdTotal, K: Exchang /// .distinct_total(); /// }); /// ``` - fn distinct_total(self) -> VecCollection { + fn distinct_total(self) -> VecCollection { self.distinct_total_core() } @@ -78,17 +78,17 @@ pub trait ThresholdTotal, K: Exchang /// This method allows `distinct` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn distinct_total_core+'static>(self) -> VecCollection { + fn distinct_total_core+'static>(self) -> VecCollection { self.threshold_total(|_,_| R2::from(1i8)) } } -impl ThresholdTotal for VecCollection +impl ThresholdTotal for VecCollection where - G: Scope, + T: Timestamp + TotalOrder + Lattice + Ord, { - fn threshold_semigroup(self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -98,21 +98,20 @@ where } } -impl ThresholdTotal for Arranged +impl ThresholdTotal for Arranged where - G: Scope, - T1: for<'a> TraceReader< + Tr: for<'a> TraceReader< Key<'a>=&'a K, Val<'a>=&'a (), Time: TotalOrder, - Diff : ExchangeData + Semigroup>, + Diff : ExchangeData + Semigroup>, >+Clone+'static, K: ExchangeData, { - fn threshold_semigroup(self, mut thresh: F) -> VecCollection + fn threshold_semigroup(self, mut thresh: F) -> VecCollection where R2: Semigroup+'static, - F: for<'a> FnMut(T1::Key<'a>,&T1::Diff,Option<&T1::Diff>)->Option+'static, + F: for<'a> FnMut(Tr::Key<'a>,&Tr::Diff,Option<&Tr::Diff>)->Option+'static, { let mut trace = self.trace.clone(); @@ -120,8 +119,8 @@ where self.stream.unary_frontier(Pipeline, "ThresholdTotal", move |_,_| { // tracks the lower and upper limit of received batches. - let mut lower_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); - let mut upper_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); + let mut lower_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); + let mut upper_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); move |(input, _frontier), output| { @@ -153,14 +152,14 @@ where let (mut trace_cursor, trace_storage) = trace.cursor_through(lower_limit.borrow()).unwrap(); while let Some(key) = batch_cursor.get_key(&batch_storage) { - let mut count: Option = None; + let mut count: Option = None; // Compute the multiplicity of this key before the current batch. trace_cursor.seek_key(&trace_storage, key); 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(Tr::owned_diff(diff)); } }); } @@ -175,7 +174,7 @@ where temp.plus_equals(&diff); thresh(key, &temp, Some(old)) }, - None => { thresh(key, &T1::owned_diff(diff), None) }, + None => { thresh(key, &Tr::owned_diff(diff), None) }, }; // Either add or assign `diff` to `count`. @@ -183,12 +182,12 @@ where count.plus_equals(&diff); } else { - count = Some(T1::owned_diff(diff)); + count = Some(Tr::owned_diff(diff)); } if let Some(difference) = difference { if !difference.is_zero() { - session.give((key.clone(), T1::owned_time(time), difference)); + session.give((key.clone(), Tr::owned_time(time), difference)); } } }); diff --git a/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index fd6aea26c..d3a3bccb8 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -4,7 +4,6 @@ use std::sync::{Arc, Mutex}; use timely::Config; -use timely::dataflow::*; use timely::dataflow::operators::Capture; use timely::dataflow::operators::capture::Extract; @@ -202,9 +201,9 @@ fn bfs_differential( } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // 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..2e48b41f3 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -7,7 +7,6 @@ use std::mem; use timely::Config; -use timely::dataflow::*; use timely::dataflow::operators::Capture; use timely::dataflow::operators::capture::Extract; @@ -215,9 +214,9 @@ fn scc_differential( .collect() } -fn _strongly_connected(graph: VecCollection) -> VecCollection +fn _strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { graph.clone().iterate(|scope, inner| { let edges = graph.enter(&scope); @@ -226,9 +225,9 @@ where }) } -fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection +fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { let nodes = edges.clone() .map_in_place(|x| x.0 = x.1) @@ -244,9 +243,9 @@ where .map(|((x1,x2),_)| (x2,x1)) } -fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection +fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { edges.clone() // <-- wth is this. .filter(|_| false) diff --git a/dogsdogsdogs/examples/delta_query.rs b/dogsdogsdogs/examples/delta_query.rs index af9eb5bfd..74d24b246 100644 --- a/dogsdogsdogs/examples/delta_query.rs +++ b/dogsdogsdogs/examples/delta_query.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; use graph_map::GraphMMap; @@ -90,7 +89,7 @@ fn main() { let changes3 = validate(changes3, reverse_self_alt.clone(), key2.clone()); let changes3 = changes3.map(|((a,c),b)| (a,b,c)); - let prev_changes = changes1.concat(changes2).concat(changes3).leave(); + let prev_changes = changes1.concat(changes2).concat(changes3).leave(&scope); // New ideas let d_edges = edges.differentiate(inner); @@ -116,7 +115,7 @@ fn main() { .join_core(forward_key_alt, |a,c,b| Some(((*c, *b), *a))) .join_core(reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c))); - let next_changes = changes1.concat(changes2).concat(changes3).integrate(); + let next_changes = changes1.concat(changes2).concat(changes3).integrate(&scope); (prev_changes, next_changes) }); diff --git a/dogsdogsdogs/examples/delta_query2.rs b/dogsdogsdogs/examples/delta_query2.rs index 6b088895e..f29a95a1a 100644 --- a/dogsdogsdogs/examples/delta_query2.rs +++ b/dogsdogsdogs/examples/delta_query2.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::order::Product; use timely::dataflow::operators::probe::Handle; use timely::dataflow::operators::vec::UnorderedInput; diff --git a/dogsdogsdogs/examples/delta_query_wcoj.rs b/dogsdogsdogs/examples/delta_query_wcoj.rs index afceef1ee..760c0afc1 100644 --- a/dogsdogsdogs/examples/delta_query_wcoj.rs +++ b/dogsdogsdogs/examples/delta_query_wcoj.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; use graph_map::GraphMMap; @@ -80,7 +79,7 @@ fn main() { ]) .map(|((a,c),b)| (a,b,c)); - changes1.concat(changes2).concat(changes3).leave() + changes1.concat(changes2).concat(changes3).leave(&scope) }); triangles diff --git a/dogsdogsdogs/examples/ngo.rs b/dogsdogsdogs/examples/ngo.rs index 340b6c988..9c1177a58 100644 --- a/dogsdogsdogs/examples/ngo.rs +++ b/dogsdogsdogs/examples/ngo.rs @@ -1,5 +1,4 @@ use std::hash::Hash; -use timely::dataflow::*; use timely::dataflow::operators::*; use timely::dataflow::operators::vec::count::Accumulate; @@ -38,9 +37,9 @@ fn main() { }).unwrap(); } -fn triangles(edges: VecCollection) -> VecCollection +fn triangles(edges: VecCollection) -> VecCollection where - G: Scope, + G: timely::progress::Timestamp + Lattice + Hash + Ord, { // 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..0ddb9572a 100644 --- a/dogsdogsdogs/src/altneu.rs +++ b/dogsdogsdogs/src/altneu.rs @@ -66,7 +66,7 @@ impl Refines for AltNeu { fn to_outer(self: AltNeu) -> T { self.time } - fn summarize(_path: ()) -> ::Summary { + fn summarize(_path: ()) -> T::Summary { Default::default() } } diff --git a/dogsdogsdogs/src/calculus.rs b/dogsdogsdogs/src/calculus.rs index 94b443231..8109d1262 100644 --- a/dogsdogsdogs/src/calculus.rs +++ b/dogsdogsdogs/src/calculus.rs @@ -13,7 +13,7 @@ //! almost everywhere empty (and so has a low memory footprint, if the system works as planned). use timely::dataflow::Scope; -use timely::dataflow::scopes::Child; +use timely::progress::Timestamp; use timely::dataflow::operators::vec::{Filter, Map}; use differential_dataflow::{AsCollection, VecCollection, Data}; use differential_dataflow::difference::Abelian; @@ -21,23 +21,23 @@ use differential_dataflow::difference::Abelian; use crate::altneu::AltNeu; /// Produce a collection containing the changes at the moments they happen. -pub trait Differentiate { - fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R>; +pub trait Differentiate { + fn differentiate(self, child: &Scope>) -> VecCollection, D, R>; } /// Collect instantaneous changes back in to a collection. -pub trait Integrate { - fn integrate(self) -> VecCollection; +pub trait Integrate { + fn integrate(self, outer: &Scope) -> VecCollection; } -impl Differentiate for VecCollection +impl Differentiate for VecCollection where - G: Scope, + T: Timestamp, D: Data, R: Abelian + 'static, { // For each (data, Alt(time), diff) we add a (data, Neu(time), -diff). - fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R> { + fn differentiate(self, child: &Scope>) -> VecCollection, D, R> { self.enter(child) .inner .flat_map(|(data, time, diff)| { @@ -51,17 +51,17 @@ where } } -impl<'a, G, D, R> Integrate for VecCollection>, D, R> +impl Integrate for VecCollection, D, R> where - G: Scope, + T: Timestamp, D: Data, R: Abelian + 'static, { // We discard each `neu` variant and strip off the `alt` wrapper. - fn integrate(self) -> VecCollection { + fn integrate(self, outer: &Scope) -> VecCollection { self.inner .filter(|(_d,t,_r)| !t.neu) .as_collection() - .leave() + .leave(outer) } } diff --git a/dogsdogsdogs/src/lib.rs b/dogsdogsdogs/src/lib.rs index f704d5988..e2699ee0d 100644 --- a/dogsdogsdogs/src/lib.rs +++ b/dogsdogsdogs/src/lib.rs @@ -1,6 +1,5 @@ use std::hash::Hash; -use timely::dataflow::Scope; use timely::progress::Timestamp; use timely::dataflow::operators::vec::Partition; use timely::dataflow::operators::Concatenate; @@ -20,37 +19,37 @@ pub mod operators; Implementors of `PrefixExtension` provide types and methods for extending a differential dataflow collection, via the three methods `count`, `propose`, and `validate`. **/ -pub trait PrefixExtender> { +pub trait PrefixExtender> { /// The required type of prefix to extend. type Prefix; /// The type to be produced as extension. type Extension; /// Annotates prefixes with the number of extensions the relation would propose. - fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection; + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection; /// Extends each prefix with corresponding extensions. - fn propose(&mut self, prefixes: VecCollection) -> VecCollection; + fn propose(&mut self, prefixes: VecCollection) -> VecCollection; /// Restricts proposed extensions by those the extender would have proposed. - fn validate(&mut self, extensions: VecCollection) -> VecCollection; + fn validate(&mut self, extensions: VecCollection) -> VecCollection; } -pub trait ProposeExtensionMethod> { - fn propose_using>(self, extender: &mut PE) -> VecCollection; - fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; +pub trait ProposeExtensionMethod> { + fn propose_using>(self, extender: &mut PE) -> VecCollection; + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; } -impl ProposeExtensionMethod for VecCollection +impl ProposeExtensionMethod for VecCollection where - G: Scope, + T: Timestamp, P: ExchangeData+Ord, R: Monoid+Multiply+'static, { - fn propose_using(self, extender: &mut PE) -> VecCollection + fn propose_using(self, extender: &mut PE) -> VecCollection where - PE: PrefixExtender + PE: PrefixExtender { extender.propose(self) } - fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection where E: ExchangeData+Ord { @@ -81,12 +80,12 @@ where } } -pub trait ValidateExtensionMethod, P, E> { - fn validate_using>(self, extender: &mut PE) -> VecCollection; +pub trait ValidateExtensionMethod, P, E> { + fn validate_using>(self, extender: &mut PE) -> VecCollection; } -impl, P, E> ValidateExtensionMethod for VecCollection { - fn validate_using>(self, extender: &mut PE) -> VecCollection { +impl, P, E> ValidateExtensionMethod for VecCollection { + fn validate_using>(self, extender: &mut PE) -> VecCollection { extender.validate(self) } } @@ -137,7 +136,7 @@ where R: Monoid+Multiply+ExchangeData, { - pub fn index>(collection: VecCollection) -> Self { + pub fn index(collection: VecCollection) -> Self { // We need to count the number of (k, v) pairs and not rely on the given Monoid R and its binary addition operation. // counts and validate can share the base arrangement let arranged = collection.clone().arrange_by_self(); @@ -180,9 +179,9 @@ where key_selector: F, } -impl PrefixExtender for CollectionExtender +impl PrefixExtender for CollectionExtender where - G: Scope, + T: Timestamp + Lattice + ExchangeData + Hash, K: ExchangeData+Hash+Default, V: ExchangeData+Hash+Default, P: ExchangeData, @@ -192,17 +191,17 @@ where type Prefix = P; type Extension = V; - fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection { + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection { let counts = self.indices.count_trace.import(&prefixes.scope()); operators::count::count(prefixes, counts, self.key_selector.clone(), index) } - fn propose(&mut self, prefixes: VecCollection) -> VecCollection { + fn propose(&mut self, prefixes: VecCollection) -> VecCollection { let propose = self.indices.propose_trace.import(&prefixes.scope()); operators::propose::propose(prefixes, propose, self.key_selector.clone()) } - fn validate(&mut self, extensions: VecCollection) -> VecCollection { + fn validate(&mut self, extensions: VecCollection) -> VecCollection { let validate = self.indices.validate_trace.import(&extensions.scope()); operators::validate::validate(extensions, validate, self.key_selector.clone()) } diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index 42367a2e1..a92066f1e 100644 --- a/dogsdogsdogs/src/operators/count.rs +++ b/dogsdogsdogs/src/operators/count.rs @@ -1,5 +1,3 @@ -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection, Hashable}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -11,14 +9,13 @@ use differential_dataflow::trace::TraceReader; /// For each triple, it extracts a key using `key_selector`, and finds the /// associated count in `arrangement`. If the found count is less than `count`, /// the `count` and `index` fields are overwritten with their new values. -pub fn count( - prefixes: VecCollection, - arrangement: Arranged, +pub fn count( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, index: usize, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: TraceReader+Clone+'static, Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, for<'a> Tr::Diff : Semigroup>, diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index 2a18a9aa1..8bedb4cbf 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -37,9 +37,11 @@ use std::time::Instant; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; -use timely::dataflow::{Scope, ScopeParent, Stream}; +use timely::dataflow::Stream; +use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::{Capability, Operator, generic::Session}; +use timely::PartialOrder; use timely::progress::Antichain; use timely::progress::frontier::AntichainRef; @@ -73,45 +75,44 @@ use differential_dataflow::trace::implementations::BatchContainer; /// Notice that the time is hoisted up into data. The expectation is that /// once out of the "delta flow region", the updates will be `delay`d to the /// times specified in the payloads. -pub fn half_join( - stream: VecCollection, - arrangement: Arranged, +pub fn half_join( + stream: VecCollection, + arrangement: Arranged, frontier_func: FF, comparison: CF, mut output_func: S, -) -> VecCollection>::Output> +) -> VecCollection>::Output> where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, R: Mul, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, + CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, DOut: Clone+'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: &Tr::Time, diff1: &R, output: &mut Vec<(Tr::Time, Tr::Diff)>| { for (time, diff2) in output.drain(..) { let diff = diff1.clone() * diff2.clone(); let dout = (output_func(k, v1, v2), time.clone()); session.give((dout, initial.clone(), diff)); } }; - half_join_internal_unsafe::<_, _, _, _, _, _,_,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) + half_join_internal_unsafe::<_, _, _, _, _, _,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) .as_collection() } -/// A session with lifetime `'a` in a scope `G` with a container builder `CB`. +/// A session with lifetime `'a` over timestamp `T` with a container builder `CB`. /// /// This is a shorthand primarily for the reson of readability. -type SessionFor<'a, 'b, G, CB> = +type SessionFor<'a, 'b, T, CB> = Session<'a, 'b, - ::Timestamp, + T, CB, - Capability<::Timestamp>, + Capability, >; /// An unsafe variant of `half_join` where the `output_func` closure takes @@ -139,25 +140,24 @@ type SessionFor<'a, 'b, G, CB> = /// yield control, as a function of the elapsed time and the number of matched /// records. Note this is not the number of *output* records, owing mainly to /// the number of matched records being easiest to record with low overhead. -pub fn half_join_internal_unsafe( - stream: VecCollection, - mut arrangement: Arranged, +pub fn half_join_internal_unsafe( + stream: VecCollection, + mut arrangement: Arranged, frontier_func: FF, comparison: CF, yield_function: Y, mut output_func: S, -) -> Stream +) -> Stream where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &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<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, { // No need to block physical merging for this operator. @@ -167,7 +167,7 @@ where let mut stash = HashMap::new(); - let exchange = Exchange::new(move |update: &((K, V, G::Timestamp),G::Timestamp,R)| (update.0).0.hashed().into()); + let exchange = Exchange::new(move |update: &((K, V, Tr::Time),Tr::Time,R)| (update.0).0.hashed().into()); // Stash for (time, diff) accumulation. let mut output_buffer = Vec::new(); @@ -215,7 +215,7 @@ where // Update yielded: We can only go from false to {false, true} as // we're checking that `!yielded` holds before entering this block. - yielded = process_proposals::( + yielded = process_proposals::<_, _, _, _, _, _, _, _>( &comparison, &yield_function, &mut output_func, @@ -247,7 +247,6 @@ where // Any remaining times should peel off elements from `proposals`. let mut additions = vec![Vec::new(); antichain.len()]; for (data, initial, diff) in proposals.drain(..) { - use timely::PartialOrder; let position = antichain.iter().position(|t| t.less_equal(&initial)).unwrap(); additions[position].push((data, initial, diff)); } @@ -300,7 +299,7 @@ where /// Leaves a zero diff in place for all proposals that were processed. /// /// Returns `true` if the operator should yield. -fn process_proposals( +fn process_proposals( comparison: &CF, yield_function: &Y, output_func: &mut S, @@ -309,16 +308,15 @@ fn process_proposals( work: &mut usize, trace: &mut Tr, proposals: &mut Vec<((K, V, Tr::Time), Tr::Time, R)>, - mut session: SessionFor, + mut session: SessionFor, frontier: AntichainRef ) -> bool where - G: Scope, Tr: TraceReader, Tr::KeyContainer: BatchContainer, CF: Fn(Tr::TimeGat<'_>, &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<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, K: Ord, V: Ord, diff --git a/dogsdogsdogs/src/operators/half_join2.rs b/dogsdogsdogs/src/operators/half_join2.rs index 08029f96e..6b19c0bbd 100644 --- a/dogsdogsdogs/src/operators/half_join2.rs +++ b/dogsdogsdogs/src/operators/half_join2.rs @@ -22,7 +22,8 @@ use std::ops::Mul; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; +use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::Operator; use timely::PartialOrder; @@ -61,27 +62,26 @@ use timely::dataflow::operators::CapabilitySet; /// Notice that the time is hoisted up into data. The expectation is that /// once out of the "delta flow region", the updates will be `delay`d to the /// times specified in the payloads. -pub fn half_join( - stream: VecCollection, - arrangement: Arranged, +pub fn half_join( + stream: VecCollection, + arrangement: Arranged, frontier_func: FF, comparison: CF, mut output_func: S, -) -> VecCollection>::Output> +) -> VecCollection>::Output> where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, R: Mul, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, + CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, DOut: Clone+'static, S: FnMut(&K, &V, Tr::Val<'_>)->DOut+'static, { - let output_func = move |builder: &mut CapacityContainerBuilder>, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &G::Timestamp, diff1: &R, output: &mut Vec<(G::Timestamp, Tr::Diff)>| { + let output_func = move |builder: &mut CapacityContainerBuilder>, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &Tr::Time, diff1: &R, output: &mut Vec<(Tr::Time, Tr::Diff)>| { for (time, diff2) in output.drain(..) { let diff = diff1.clone() * diff2.clone(); let dout = (output_func(k, v1, v2), time.clone()); @@ -89,7 +89,7 @@ where builder.push_into((dout, initial.clone(), diff)); } }; - half_join_internal_unsafe::<_, _, _, _, _, _,_,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) + half_join_internal_unsafe::<_, _, _, _, _, _,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) .as_collection() } @@ -118,25 +118,24 @@ where /// yield control, as a function of the elapsed time and the number of matched /// records. Note this is not the number of *output* records, owing mainly to /// the number of matched records being easiest to record with low overhead. -pub fn half_join_internal_unsafe( - stream: VecCollection, - mut arrangement: Arranged, +pub fn half_join_internal_unsafe( + stream: VecCollection, + mut arrangement: Arranged, frontier_func: FF, comparison: CF, yield_function: Y, mut output_func: S, -) -> Stream +) -> Stream where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, - S: FnMut(&mut CB, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut CB, &K, &V, Tr::Val<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, { // No need to block physical merging for this operator. @@ -144,7 +143,7 @@ where let mut arrangement_trace = Some(arrangement.trace); let arrangement_stream = arrangement.stream; - let exchange = Exchange::new(move |update: &((K, V, G::Timestamp),G::Timestamp,R)| (update.0).0.hashed().into()); + let exchange = Exchange::new(move |update: &((K, V, Tr::Time),Tr::Time,R)| (update.0).0.hashed().into()); // Stash for (time, diff) accumulation. let mut output_buffer = Vec::new(); @@ -152,7 +151,7 @@ where // Unified blobs: each blob holds data in (T, D, R) order, with a stuck_count // tracking how many elements at the back are not yet eligible for processing. // The ready prefix is sorted by (D, T, R) for cursor traversal. - let mut blobs: Vec> = Vec::new(); + let mut blobs: Vec> = Vec::new(); let scope = stream.scope(); stream.inner.binary_frontier(arrangement_stream, exchange, Pipeline, "HalfJoin", move |_,info| { @@ -163,7 +162,7 @@ where move |(input1, frontier1), (input2, frontier2), output| { // Drain all input into a single buffer. - let mut arriving: Vec<(G::Timestamp, (K, V, G::Timestamp), R)> = Vec::new(); + let mut arriving: Vec<(Tr::Time, (K, V, Tr::Time), R)> = Vec::new(); let mut caps = CapabilitySet::new(); input1.for_each(|capability, data| { caps.insert(capability.retain(0)); @@ -189,7 +188,7 @@ where if let Some(min_time) = frontier.iter().min() { time_con.push_own(min_time); } - let eligible = |initial: &G::Timestamp| -> bool { + let eligible = |initial: &Tr::Time| -> bool { !(0..time_con.len()).any(|i| comparison(time_con.index(i), initial)) }; @@ -265,7 +264,7 @@ where let (mut cursor, storage) = trace.cursor(); let mut key_con = Tr::KeyContainer::with_capacity(1); - let mut removals: ChangeBatch = ChangeBatch::new(); + let mut removals: ChangeBatch = ChangeBatch::new(); // Process ready elements from the front. while blob.data.len() > blob.stuck_count { diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index 65108ed59..b15aba314 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -1,9 +1,8 @@ use std::collections::HashMap; -use timely::PartialOrder; -use timely::dataflow::Scope; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::Operator; +use timely::PartialOrder; use timely::progress::Antichain; use differential_dataflow::{ExchangeData, VecCollection, AsCollection, Hashable}; @@ -17,17 +16,16 @@ use differential_dataflow::trace::implementations::BatchContainer; /// This method takes a stream of prefixes and for each determines a /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. -pub fn lookup_map( - prefixes: VecCollection, - mut arrangement: Arranged, +pub fn lookup_map( + prefixes: VecCollection, + mut arrangement: Arranged, key_selector: F, mut output_func: S, supplied_key0: K, supplied_key1: K, supplied_key2: K, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< Time: std::hash::Hash, Diff : Semigroup>+Monoid+ExchangeData, @@ -51,7 +49,7 @@ where let mut logic2 = key_selector.clone(); let mut key: K = supplied_key0; - let exchange = Exchange::new(move |update: &(D,G::Timestamp,R)| { + let exchange = Exchange::new(move |update: &(D,Tr::Time,R)| { logic1(&update.0, &mut key); key.hashed().into() }); diff --git a/dogsdogsdogs/src/operators/propose.rs b/dogsdogsdogs/src/operators/propose.rs index 6cdcb130d..869e67aab 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -1,5 +1,3 @@ -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection, Hashable}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -13,13 +11,12 @@ use differential_dataflow::trace::TraceReader; /// create a join if the `prefixes` collection is also arranged and responds to changes that /// `arrangement` undergoes. More complicated patterns are also appropriate, as in the case /// of delta queries. -pub fn propose( - prefixes: VecCollection, - arrangement: Arranged, +pub fn propose( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< ValOwn = V, Time: std::hash::Hash, @@ -47,13 +44,12 @@ where /// Unlike `propose`, this method does not scale the multiplicity of matched /// prefixes by the number of matches in `arrangement`. This can be useful to /// avoid the need to prepare an arrangement of distinct extensions. -pub fn propose_distinct( - prefixes: VecCollection, - arrangement: Arranged, +pub fn propose_distinct( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< ValOwn = V, Time: std::hash::Hash, diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index 0ab7297bb..0a28c2483 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -1,7 +1,5 @@ use std::hash::Hash; -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -12,13 +10,12 @@ use differential_dataflow::trace::TraceReader; /// This method takes a stream of prefixes and for each determines a /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. -pub fn validate( - extensions: VecCollection, - arrangement: Arranged, +pub fn validate( + extensions: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< Time: std::hash::Hash, Diff : Semigroup>+Monoid+Multiply+ExchangeData, diff --git a/experiments/Cargo.toml b/experiments/Cargo.toml index edcf55fb3..ff46dd329 100644 --- a/experiments/Cargo.toml +++ b/experiments/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] core_affinity = "0.5.9" rand="0.3.13" -timely = { workspace = true } +timely = { workspace = true, features = ["getopts"] } differential-dataflow = { workspace = true } graph_map = { git = "https://github.com/frankmcsherry/graph-map" } serde = { version = "1.0", features = ["derive"] } diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 3ea53cd14..a3c29b5e3 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -1,7 +1,6 @@ use std::time::Instant; use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::WorkerConfig; use differential_dataflow::input::Input; @@ -13,7 +12,7 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; type Node = u32; @@ -204,13 +203,12 @@ fn main() { }).unwrap(); } -fn interactive( +fn interactive( edges: Arrange, tc_1: VecCollection, tc_2: VecCollection, sg_x: VecCollection -) -> VecCollection -where G::Timestamp: Lattice{ +) -> VecCollection { // descendants of tc_1: let tc_1_enter = tc_1.clone(); diff --git a/experiments/src/bin/deals.rs b/experiments/src/bin/deals.rs index 782941e15..89c9699a0 100644 --- a/experiments/src/bin/deals.rs +++ b/experiments/src/bin/deals.rs @@ -1,7 +1,5 @@ use std::time::Instant; -use timely::dataflow::*; - use differential_dataflow::input::Input; use differential_dataflow::VecCollection; use differential_dataflow::operators::*; @@ -11,9 +9,10 @@ use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::operators::arrange::Arrange; use differential_dataflow::operators::iterate::Variable; +use differential_dataflow::lattice::Lattice; use differential_dataflow::difference::Present; -type EdgeArranged = Arranged::Timestamp, R>>>; +type EdgeArranged = Arranged>>; type Node = u32; type Edge = (Node, Node); @@ -83,10 +82,11 @@ fn main() { use timely::order::Product; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn tc>(edges: EdgeArranged) -> VecCollection { +fn tc(edges: EdgeArranged) -> VecCollection { // repeatedly update minimal distances each node can be reached from each root - edges.stream.scope().iterative::(|scope| { + let outer = edges.stream.scope(); + outer.iterative::(|scope| { let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let edges = edges.enter(scope); @@ -102,18 +102,19 @@ fn tc>(edges: EdgeArranged) -> Ve ; inner.set(result.clone()); - result.leave() + result.leave(&outer) } ) } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn sg>(edges: EdgeArranged) -> VecCollection { +fn sg(edges: EdgeArranged) -> VecCollection { let peers = edges.clone().join_core(edges.clone(), |_,&x,&y| Some((x,y))).filter(|&(x,y)| x != y); // repeatedly update minimal distances each node can be reached from each root - peers.scope().iterative::(|scope| { + let outer = peers.scope(); + outer.iterative::(|scope| { let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let edges = edges.enter(scope); @@ -131,7 +132,7 @@ fn sg>(edges: EdgeArranged) -> Ve ; inner.set(result.clone()); - result.leave() + result.leave(&outer) } ) } diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 6e5b0ffaa..0ba7ddd4b 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -2,7 +2,7 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -258,14 +258,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -288,13 +288,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -356,13 +357,14 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } -fn connected_components(graph: Arrange) -> VecCollection -where G::Timestamp: Lattice + std::hash::Hash { +fn connected_components(graph: Arrange) -> VecCollection +where T: Lattice + std::hash::Hash +{ // 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..0a5ee0d25 100644 --- a/experiments/src/bin/graphs-interactive-neu-zwei.rs +++ b/experiments/src/bin/graphs-interactive-neu-zwei.rs @@ -1,6 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -226,14 +226,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -256,12 +256,12 @@ where G::Timestamp: Lattice+Ord { } // // returns pairs (n, s) indicating node n can be reached from a root in s steps. -// fn bidijkstra( -// forward_graph: &Arrange, -// reverse_graph: &Arrange, -// goals: VecCollection, -// bound: u64) -> VecCollection -// where G::Timestamp: Lattice+Ord { +// fn bidijkstra( +// forward_graph: &Arrange, +// reverse_graph: &Arrange, +// goals: VecCollection, +// bound: u64) -> VecCollection +// where T: Lattice+Ord { // goals.scope().scoped(|inner| { diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index 7e27e7ed8..ceda7913c 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -2,7 +2,7 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -291,14 +291,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -321,13 +321,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -389,6 +390,6 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index 2ec74e93f..9ee11bf7b 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -1,6 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -194,15 +194,15 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -225,13 +225,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -293,6 +294,6 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/experiments/src/bin/graphs-static.rs b/experiments/src/bin/graphs-static.rs index 16cbbd6dd..46a89b4a2 100644 --- a/experiments/src/bin/graphs-static.rs +++ b/experiments/src/bin/graphs-static.rs @@ -1,5 +1,3 @@ -use timely::dataflow::*; - use timely::order::Product; use timely::dataflow::operators::ToStream; @@ -106,14 +104,15 @@ use differential_dataflow::operators::arrange::TraceAgent; type TraceHandle = TraceAgent; -fn reach> ( +fn reach( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node, Diff> +) -> VecCollection<(), Node, Diff> { let graph = graph.import(&roots.scope()); - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { let graph = graph.enter(scope); let roots = roots.enter(scope); @@ -126,20 +125,21 @@ fn reach> ( .threshold_total(|_,_| 1); inner.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn bfs> ( +fn bfs( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node, Diff> +) -> VecCollection<(), (Node, u32), Diff> { let graph = graph.import(&roots.scope()); let roots = roots.map(|r| (r,0)); - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { let graph = graph.enter(scope); let roots = roots.enter(scope); @@ -151,15 +151,15 @@ fn bfs> ( .reduce(|_key, input, output| output.push((*input[0].0,1))); inner.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn connected_components>( - scope: &mut G, +fn connected_components( + scope: &mut timely::dataflow::Scope<()>, forward: &mut TraceHandle, reverse: &mut TraceHandle, -) -> VecCollection { +) -> VecCollection<(), (Node, Node), Diff> { let forward = forward.import(scope); let reverse = reverse.import(scope); @@ -169,14 +169,14 @@ fn connected_components>( let nodes_r = reverse.clone().flat_map_ref(|k,v| if k < v { Some(*k) } else { None }); let nodes = nodes_f.concat(nodes_r).consolidate().map(|x| (x,x)); - scope.iterative(|scope| { + scope.iterative(|inner_scope| { // import arrangements, nodes. - let forward = forward.enter(scope); - let reverse = reverse.enter(scope); - let nodes = nodes.enter(scope); + let forward = forward.enter(inner_scope); + let reverse = reverse.enter(inner_scope); + let nodes = nodes.enter(inner_scope); - let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (inner, inner_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let labels = inner_collection.clone().arrange_by_key(); let f_prop = labels.clone().join_core(forward, |_k,l,d| Some((*d,*l))); @@ -197,6 +197,6 @@ fn connected_components>( .reduce(|_, s, t| { t.push((*s[0].0, 1)); }); inner.set(result.clone()); - result.leave() + result.leave(scope) }) } diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index ea7420120..7461d006a 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -1,7 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; - use differential_dataflow::input::Input; use differential_dataflow::VecCollection; use differential_dataflow::operators::*; @@ -88,10 +86,10 @@ use differential_dataflow::operators::arrange::TraceAgent; type TraceHandle = TraceAgent; -fn reach> ( +fn reach( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node> +) -> VecCollection<(), Node> { let graph = graph.import(&roots.scope()); @@ -110,10 +108,10 @@ fn reach> ( } -fn bfs> ( +fn bfs( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node> +) -> VecCollection<(), (Node, u32)> { let graph = graph.import(&roots.scope()); let roots = roots.map(|r| (r,0)); @@ -129,9 +127,9 @@ fn bfs> ( }) } -// fn connected_components>( +// fn connected_components( // graph: &mut TraceHandle -// ) -> VecCollection { +// ) -> VecCollection<(), (Node, Node)> { // // 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/graspan1.rs b/experiments/src/bin/graspan1.rs index 507f5c46b..571b46f1b 100644 --- a/experiments/src/bin/graspan1.rs +++ b/experiments/src/bin/graspan1.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::Scope; use timely::order::Product; use differential_dataflow::difference::Present; @@ -35,8 +34,9 @@ fn main() { // a N c <- a N b && b E c // N(a,c) <- N(a,b), E(b, c) + let outer = nodes.scope(); let reached = - nodes.scope().iterative::(|inner| { + outer.iterative::(|inner| { let nodes = nodes.enter(inner).map(|(a,b)| (b,a)); let edges = edges.enter(inner); @@ -51,7 +51,7 @@ fn main() { .threshold_semigroup(|_,_,x: Option<&Present>| if x.is_none() { Some(Present) } else { None }); labels.set(next.clone()); - next.leave() + next.leave(&outer) }); reached diff --git a/experiments/src/bin/graspan2.rs b/experiments/src/bin/graspan2.rs index 4ce8574a9..f101ae246 100644 --- a/experiments/src/bin/graspan2.rs +++ b/experiments/src/bin/graspan2.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::Scope; use timely::order::Product; use differential_dataflow::operators::iterate::Variable; @@ -51,14 +50,14 @@ fn unoptimized() { let (value_flow, memory_alias, value_alias) = scope - .iterative::(|scope| { + .iterative::(|inner_scope| { - let nodes = nodes.enter(scope); - let assignment = assignment.enter(scope); - let dereference = dereference.enter(scope); + let nodes = nodes.enter(inner_scope); + let assignment = assignment.enter(inner_scope); + let dereference = dereference.enter(inner_scope); - let (value_flow, value_flow_collection) = Variable::new(scope, Product::new(Default::default(), 1)); - let (memory_alias, memory_alias_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (value_flow, value_flow_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); + let (memory_alias, memory_alias_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let value_flow_arranged = value_flow_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); let memory_alias_arranged = memory_alias_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -108,7 +107,7 @@ fn unoptimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(), memory_alias_next.leave(), value_alias_next.leave()) + (value_flow_next.leave(&scope), memory_alias_next.leave(&scope), value_alias_next.leave(&scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); @@ -176,14 +175,14 @@ fn optimized() { let (value_flow, memory_alias) = scope - .iterative::(|scope| { + .iterative::(|inner_scope| { - let nodes = nodes.enter(scope); - let assignment = assignment.enter(scope); - let dereference = dereference.enter(scope); + let nodes = nodes.enter(inner_scope); + let assignment = assignment.enter(inner_scope); + let dereference = dereference.enter(inner_scope); - let (value_flow, value_flow_collection) = Variable::new(scope, Product::new(Default::default(), 1)); - let (memory_alias, memory_alias_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (value_flow, value_flow_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); + let (memory_alias, memory_alias_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let value_flow_arranged = value_flow_collection.clone().arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); let memory_alias_arranged = memory_alias_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -233,7 +232,7 @@ fn optimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(), memory_alias_next.leave()) + (value_flow_next.leave(&scope), memory_alias_next.leave(&scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); diff --git a/interactive/src/command.rs b/interactive/src/command.rs index 5d110ea70..ba83fca60 100644 --- a/interactive/src/command.rs +++ b/interactive/src/command.rs @@ -4,7 +4,6 @@ use std::hash::Hash; use std::io::Write; use serde::{Deserialize, Serialize}; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEvent; @@ -48,7 +47,7 @@ where { /// Executes a command. - pub fn execute(self, manager: &mut Manager, worker: &mut Worker) { + pub fn execute(self, manager: &mut Manager, worker: &mut Worker) { match self { diff --git a/interactive/src/logging.rs b/interactive/src/logging.rs index 322dd16d2..0b076f36f 100644 --- a/interactive/src/logging.rs +++ b/interactive/src/logging.rs @@ -3,7 +3,6 @@ use std::hash::Hash; use std::time::Duration; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEvent; use timely::dataflow::operators::capture::event::EventIterator; @@ -20,16 +19,15 @@ pub trait LoggingValue : VectorFrom+VectorFrom { impl+VectorFrom> LoggingValue for V { } /// Timely logging capture and arrangement. -pub fn publish_timely_logging( +pub fn publish_timely_logging( manager: &mut Manager, - worker: &mut Worker, + worker: &mut Worker, granularity_ns: u64, name: &str, events: I ) where V: ExchangeData+Hash+LoggingValue+Datum, - A: Allocate, I : IntoIterator, ::Item: EventIterator>+'static { @@ -209,16 +207,15 @@ where } /// Timely logging capture and arrangement. -pub fn publish_differential_logging( +pub fn publish_differential_logging( manager: &mut Manager, - worker: &mut Worker, + worker: &mut Worker, granularity_ns: u64, name: &str, events: I ) where V: ExchangeData+Hash+LoggingValue+Datum, - A: Allocate, I : IntoIterator, ::Item: EventIterator>+'static { diff --git a/interactive/src/manager.rs b/interactive/src/manager.rs index 8579a538e..f9a0f3872 100644 --- a/interactive/src/manager.rs +++ b/interactive/src/manager.rs @@ -5,7 +5,6 @@ use std::hash::Hash; // use std::time::Duration; use timely::dataflow::ProbeHandle; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEventBuilder; @@ -79,7 +78,7 @@ impl Manager // } /// Clear the managed inputs and traces. - pub fn shutdown(&mut self, worker: &mut Worker) { + pub fn shutdown(&mut self, worker: &mut Worker) { self.inputs.sessions.clear(); self.traces.inputs.clear(); self.traces.arrangements.clear(); diff --git a/interactive/src/plan/concat.rs b/interactive/src/plan/concat.rs index 4c27941bb..149b81049 100644 --- a/interactive/src/plan/concat.rs +++ b/interactive/src/plan/concat.rs @@ -19,10 +19,10 @@ impl Render for Concat { type Value = V; - fn render>( + fn render( &self, - scope: &mut S, - arrangements: &mut TraceManager) -> VecCollection, Diff> + scope: &mut Scope) { - - // use std::rc::Rc; - // use timely::dataflow::operators::capture::event::link::EventLink; - // use timely::logging::BatchLogger; - - // let timely_events = Rc::new(EventLink::new()); - // let differential_events = Rc::new(EventLink::new()); - - // self.publish_timely_logging(worker, Some(timely_events.clone())); - // self.publish_differential_logging(worker, Some(differential_events.clone())); - - // let mut timely_logger = BatchLogger::new(timely_events.clone()); - // worker - // .log_register() - // .insert::("timely", move |time, data| timely_logger.publish_batch(time, data)); - - // let mut differential_logger = BatchLogger::new(differential_events.clone()); - // worker - // .log_register() - // .insert::("differential/arrange", move |time, data| differential_logger.publish_batch(time, data)); - - // } - - /// Clear the managed inputs and traces. - pub fn shutdown(&mut self, worker: &mut Worker) { - self.inputs.sessions.clear(); - self.traces.inputs.clear(); - self.traces.arrangements.clear(); - - // Deregister loggers, so that the logging dataflows can shut down. - worker - .log_register() - .map(|mut log_register| log_register.insert::("timely", move |_time, _data| { })); - - worker - .log_register() - .map(|mut log_register| log_register.insert::("differential/arrange", move |_time, _data| { })); - } - - /// Inserts a new input session by name. - pub fn insert_input( - &mut self, - name: String, - input: InputSession, Diff>, - trace: KeysOnlyHandle) - { - self.inputs.sessions.insert(name.clone(), input); - self.traces.set_unkeyed(&Plan::Source(name), &trace); - } - - /// Advances inputs and traces to `time`. - pub fn advance_time(&mut self, time: &Time) { - self.inputs.advance_time(time); - self.traces.advance_time(time); - } - - // /// Timely logging capture and arrangement. - // pub fn publish_timely_logging(&mut self, worker: &mut Worker, events: I) - // where - // A: Allocate, - // I : IntoIterator, - // ::Item: EventIterator+'static - // { - // crate::logging::publish_timely_logging(self, worker, 1, "interactive", events) - // } - - // /// Timely logging capture and arrangement. - // pub fn publish_differential_logging(&mut self, worker: &mut Worker, events: I) - // where - // A: Allocate, - // I : IntoIterator, - // ::Item: EventIterator+'static - // { - // crate::logging::publish_differential_logging(self, worker, 1, "interactive", events) - // } -} - -/// Manages input sessions. -pub struct InputManager { - /// Input sessions by name. - pub sessions: HashMap, Diff>>, -} - -impl InputManager { - - /// Creates a new empty input manager. - pub fn new() -> Self { Self { sessions: HashMap::new() } } - - /// Advances the times of all managed inputs. - pub fn advance_time(&mut self, time: &Time) { - for session in self.sessions.values_mut() { - session.advance_to(time.clone()); - session.flush(); - } - } - -} - -/// Root handles to maintained collections. -/// -/// Manages a map from plan (describing a collection) -/// to various arranged forms of that collection. -pub struct TraceManager { - - /// Arrangements where the record itself is they key. - /// - /// This contains both input collections, which are here cached so that - /// they can be re-used, intermediate collections that are cached, and - /// any collections that are explicitly published. - inputs: HashMap, KeysOnlyHandle>, - - /// Arrangements of collections by key. - arrangements: HashMap, HashMap, KeysValsHandle>>, - -} - -impl TraceManager { - - /// Creates a new empty trace manager. - pub fn new() -> Self { - Self { - inputs: HashMap::new(), - arrangements: HashMap::new() - } - } - - /// Advances the frontier of each maintained trace. - pub fn advance_time(&mut self, time: &Time) { - use differential_dataflow::trace::TraceReader; - use timely::progress::frontier::Antichain; - let frontier = Antichain::from_elem(time.clone()); - for trace in self.inputs.values_mut() { - trace.set_logical_compaction(frontier.borrow()); - trace.set_physical_compaction(frontier.borrow()); - } - for map in self.arrangements.values_mut() { - for trace in map.values_mut() { - trace.set_logical_compaction(frontier.borrow()); - trace.set_physical_compaction(frontier.borrow()); - } - } - } - - /// Recover an arrangement by plan and keys, if it is cached. - pub fn get_unkeyed(&self, plan: &Plan) -> Option> { - self.inputs - .get(plan) - .map(|x| x.clone()) - } - - /// Installs an unkeyed arrangement for a specified plan. - pub fn set_unkeyed(&mut self, plan: &Plan, handle: &KeysOnlyHandle) { - self.inputs - .insert(plan.clone(), handle.clone()); - } - - /// Recover an arrangement by plan and keys, if it is cached. - pub fn get_keyed(&self, plan: &Plan, keys: &[usize]) -> Option> { - self.arrangements - .get(plan) - .and_then(|map| map.get(keys).map(|x| x.clone())) - } - - /// Installs a keyed arrangement for a specified plan and sequence of keys. - pub fn set_keyed(&mut self, plan: &Plan, keys: &[usize], handle: &KeysValsHandle) { - self.arrangements - .entry(plan.clone()) - .or_insert(HashMap::new()) - .insert(keys.to_vec(), handle.clone()); - } - -} diff --git a/interactive/src/parse/applicative.rs b/interactive/src/parse/applicative.rs new file mode 100644 index 000000000..b284777ef --- /dev/null +++ b/interactive/src/parse/applicative.rs @@ -0,0 +1,161 @@ +//! Applicative syntax parser for .ddir files. +//! +//! Syntax: `MAP(e, proj)`, `JOIN(e1, e2, proj)`, `REDUCE(e, MIN)`, etc. + +use super::*; + +#[derive(Debug, Clone, PartialEq)] +enum Token { + Let, Var, Scope, Result, + Input, Map, Join, Reduce, Concat, Arrange, Filter, Negate, EnterAt, Inspect, + Min, Distinct, Count, + Ident(String), Int(i64), + Dollar, LParen, RParen, LBrace, RBrace, LBracket, RBracket, + Comma, Semi, Colon, ColonColon, Eq, EqEq, NotEq, Lt, LtEq, Gt, GtEq, + Plus, Minus, Star, Eof, +} + +fn tokenize(input: &str) -> Vec { + let mut tokens = Vec::new(); + let mut chars = input.chars().peekable(); + while let Some(&ch) = chars.peek() { + match ch { + ' ' | '\t' | '\n' | '\r' => { chars.next(); }, + '-' if chars.clone().nth(1).map_or(false, |c| c == '-') => { + while let Some(&c) = chars.peek() { chars.next(); if c == '\n' { break; } } + }, + '(' => { chars.next(); tokens.push(Token::LParen); }, + ')' => { chars.next(); tokens.push(Token::RParen); }, + '{' => { chars.next(); tokens.push(Token::LBrace); }, + '}' => { chars.next(); tokens.push(Token::RBrace); }, + '[' => { chars.next(); tokens.push(Token::LBracket); }, + ']' => { chars.next(); tokens.push(Token::RBracket); }, + ',' => { chars.next(); tokens.push(Token::Comma); }, + ';' => { chars.next(); tokens.push(Token::Semi); }, + '=' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::EqEq); } else { tokens.push(Token::Eq); } }, + '!' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::NotEq); } else { panic!("Expected != after !"); } }, + '<' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::LtEq); } else { tokens.push(Token::Lt); } }, + '>' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::GtEq); } else { tokens.push(Token::Gt); } }, + '+' => { chars.next(); tokens.push(Token::Plus); }, + '*' => { chars.next(); tokens.push(Token::Star); }, + '$' => { chars.next(); tokens.push(Token::Dollar); }, + '-' => { chars.next(); tokens.push(Token::Minus); }, + ':' => { chars.next(); if chars.peek() == Some(&':') { chars.next(); tokens.push(Token::ColonColon); } else { tokens.push(Token::Colon); } }, + c if c.is_ascii_digit() => { + let mut num = String::new(); + while let Some(&c) = chars.peek() { if c.is_ascii_digit() { num.push(c); chars.next(); } else { break; } } + tokens.push(Token::Int(num.parse().unwrap())); + }, + c if c.is_ascii_alphabetic() || c == '_' => { + let mut ident = String::new(); + while let Some(&c) = chars.peek() { if c.is_ascii_alphanumeric() || c == '_' { ident.push(c); chars.next(); } else { break; } } + tokens.push(match ident.as_str() { + "let" => Token::Let, "var" => Token::Var, "scope" => Token::Scope, "result" => Token::Result, + "INPUT" => Token::Input, "MAP" => Token::Map, "JOIN" => Token::Join, + "REDUCE" => Token::Reduce, "CONCAT" => Token::Concat, "ARRANGE" => Token::Arrange, + "FILTER" => Token::Filter, "NEGATE" => Token::Negate, "ENTER_AT" => Token::EnterAt, "INSPECT" => Token::Inspect, + "MIN" => Token::Min, "DISTINCT" => Token::Distinct, "COUNT" => Token::Count, + _ => Token::Ident(ident), + }); + }, + other => panic!("Unexpected character: {:?}", other), + } + } + tokens.push(Token::Eof); + tokens +} + +struct Parser { tokens: Vec, pos: usize } + +impl Parser { + fn new(tokens: Vec) -> Self { Parser { tokens, pos: 0 } } + fn peek(&self) -> &Token { &self.tokens[self.pos] } + fn next(&mut self) -> Token { let t = self.tokens[self.pos].clone(); self.pos += 1; t } + fn expect(&mut self, expected: &Token) { let t = self.next(); assert_eq!(&t, expected, "Expected {:?}, got {:?}", expected, t); } + + fn parse_program(&mut self) -> Vec { + let mut stmts = Vec::new(); + while *self.peek() != Token::Eof && *self.peek() != Token::RBrace { stmts.push(self.parse_stmt()); } + stmts + } + + fn parse_stmt(&mut self) -> Stmt { + match self.peek().clone() { + Token::Let => { self.next(); let n = self.parse_ident(); self.expect(&Token::Eq); let e = self.parse_expr(); self.expect(&Token::Semi); Stmt::Let(n, e) }, + Token::Var => { self.next(); let n = self.parse_ident(); self.expect(&Token::Eq); let e = self.parse_expr(); self.expect(&Token::Semi); Stmt::Var(n, e) }, + Token::Result => { self.next(); let e = self.parse_expr(); self.expect(&Token::Semi); Stmt::Result(e) }, + Token::Ident(_) => { + let n = self.parse_ident(); self.expect(&Token::Colon); + if *self.peek() == Token::Scope { self.next(); } + self.expect(&Token::LBrace); let b = self.parse_program(); self.expect(&Token::RBrace); Stmt::Scope(n, b) + }, + other => panic!("Unexpected token: {:?}", other), + } + } + + fn parse_ident(&mut self) -> String { match self.next() { Token::Ident(s) => s, other => panic!("Expected ident, got {:?}", other) } } + + fn parse_expr(&mut self) -> Expr { + match self.peek().clone() { + Token::Input => { self.next(); match self.next() { Token::Int(n) => Expr::Input(n as usize), o => panic!("Expected int, got {:?}", o) } }, + Token::Map => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::Comma); let p = self.parse_projection(); self.expect(&Token::RParen); Expr::Map(Box::new(i), p) }, + Token::Join => { self.next(); self.expect(&Token::LParen); let l = self.parse_expr(); self.expect(&Token::Comma); let r = self.parse_expr(); self.expect(&Token::Comma); let p = self.parse_projection(); self.expect(&Token::RParen); Expr::Join(Box::new(l), Box::new(r), p) }, + Token::Reduce => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::Comma); let r = self.parse_reducer(); self.expect(&Token::RParen); Expr::Reduce(Box::new(i), r) }, + Token::Filter => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::Comma); let c = self.parse_condition(); self.expect(&Token::RParen); Expr::Filter(Box::new(i), c) }, + Token::Negate => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::RParen); Expr::Negate(Box::new(i)) }, + Token::EnterAt => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::Comma); let f = self.parse_field(); self.expect(&Token::RParen); Expr::EnterAt(Box::new(i), f) }, + Token::Inspect => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::Comma); let label = self.parse_ident(); self.expect(&Token::RParen); Expr::Inspect(Box::new(i), label) }, + Token::Concat => { self.next(); self.expect(&Token::LParen); let mut v = vec![self.parse_expr()]; while *self.peek() == Token::Comma { self.next(); v.push(self.parse_expr()); } self.expect(&Token::RParen); Expr::Concat(v) }, + Token::Arrange => { self.next(); self.expect(&Token::LParen); let i = self.parse_expr(); self.expect(&Token::RParen); Expr::Arrange(Box::new(i)) }, + Token::Ident(_) => { let n = self.parse_ident(); if *self.peek() == Token::ColonColon { self.next(); let f = self.parse_ident(); Expr::Qualified(n, f) } else { Expr::Name(n) } }, + other => panic!("Unexpected token in expr: {:?}", other), + } + } + + fn parse_projection(&mut self) -> Projection { + self.expect(&Token::LParen); + if *self.peek() == Token::RParen { self.next(); return Projection { key: vec![], val: vec![] }; } + if *self.peek() == Token::Semi { self.next(); + if *self.peek() == Token::RParen { self.next(); return Projection { key: vec![], val: vec![] }; } + let mut val = vec![self.parse_field()]; + while *self.peek() == Token::Comma { self.next(); val.push(self.parse_field()); } + self.expect(&Token::RParen); + return Projection { key: vec![], val }; + } + let mut key = vec![self.parse_field()]; + while *self.peek() == Token::Comma { self.next(); key.push(self.parse_field()); } + let val = if *self.peek() == Token::Semi { self.next(); + if *self.peek() == Token::RParen { vec![] } + else { let mut v = vec![self.parse_field()]; while *self.peek() == Token::Comma { self.next(); v.push(self.parse_field()); } v } + } else { vec![] }; + self.expect(&Token::RParen); + Projection { key, val } + } + + fn parse_field(&mut self) -> FieldExpr { + match self.peek().clone() { + Token::Dollar => { self.next(); let n = match self.next() { Token::Int(n) => n as usize, o => panic!("Expected int, got {:?}", o) }; if *self.peek() == Token::LBracket { self.next(); let i = match self.next() { Token::Int(i) => i as usize, o => panic!("Expected int, got {:?}", o) }; self.expect(&Token::RBracket); FieldExpr::Index(n, i) } else { FieldExpr::Pos(n) } }, + Token::Minus => { self.next(); FieldExpr::Neg(Box::new(self.parse_field())) }, + Token::Int(n) => { self.next(); FieldExpr::Const(n) }, + other => panic!("Unexpected token in field: {:?}", other), + } + } + + fn parse_condition(&mut self) -> Condition { + let l = self.parse_field(); + let op = self.next(); + let r = self.parse_field(); + match op { + Token::EqEq => Condition::Eq(l, r), + Token::NotEq => Condition::Ne(l, r), + Token::Lt => Condition::Lt(l, r), + Token::LtEq => Condition::Le(l, r), + Token::Gt => Condition::Gt(l, r), + Token::GtEq => Condition::Ge(l, r), + o => panic!("Expected comparison operator, got {:?}", o), + } + } + fn parse_reducer(&mut self) -> Reducer { match self.next() { Token::Min => Reducer::Min, Token::Distinct => Reducer::Distinct, Token::Count => Reducer::Count, o => panic!("Expected reducer, got {:?}", o) } } +} + +pub fn parse(input: &str) -> Vec { let tokens = tokenize(input); let mut p = Parser::new(tokens); p.parse_program() } diff --git a/interactive/src/parse/mod.rs b/interactive/src/parse/mod.rs new file mode 100644 index 000000000..26c4bc437 --- /dev/null +++ b/interactive/src/parse/mod.rs @@ -0,0 +1,49 @@ +//! Concrete syntax for DD IR programs. +//! +//! Two front-ends that produce the same AST: +//! - `applicative::parse()` — S-expression-like syntax (MAP, JOIN, etc.) for .ddir files +//! - `pipe::parse()` — pipe-oriented syntax (`expr | op | op`) for .ddp files + +pub mod applicative; +pub mod pipe; + +#[derive(Debug, Clone)] +pub enum FieldExpr { + Pos(usize), + Index(usize, usize), + Const(i64), + Neg(Box), +} + +#[derive(Debug, Clone)] +pub enum Condition { Eq(FieldExpr, FieldExpr), Ne(FieldExpr, FieldExpr), Lt(FieldExpr, FieldExpr), Le(FieldExpr, FieldExpr), Gt(FieldExpr, FieldExpr), Ge(FieldExpr, FieldExpr) } + +#[derive(Debug, Clone)] +pub struct Projection { pub key: Vec, pub val: Vec } + +#[derive(Debug, Clone)] +pub enum Reducer { Min, Distinct, Count } + +#[derive(Debug, Clone)] +pub enum Expr { + Input(usize), + Name(String), + Qualified(String, String), + Map(Box, Projection), + Join(Box, Box, Projection), + Reduce(Box, Reducer), + Filter(Box, Condition), + Negate(Box), + EnterAt(Box, FieldExpr), + Inspect(Box, String), + Concat(Vec), + Arrange(Box), +} + +#[derive(Debug)] +pub enum Stmt { + Let(String, Expr), + Var(String, Expr), + Scope(String, Vec), + Result(Expr), +} diff --git a/interactive/src/parse/pipe.rs b/interactive/src/parse/pipe.rs new file mode 100644 index 000000000..4a6feb49e --- /dev/null +++ b/interactive/src/parse/pipe.rs @@ -0,0 +1,199 @@ +//! Pipe syntax parser for .ddp files. +//! +//! Syntax: `expr | key(proj)`, `expr | join(e2, proj)`, `a + b`, `a - b`, etc. + +use super::*; + +#[derive(Debug, Clone, PartialEq)] +enum Token { + Let, Var, Result, + Input, Key, Map, Join, Min, Distinct, Count, Arrange, Negate, Filter, EnterAt, Inspect, + Ident(String), Int(i64), + Dollar, LParen, RParen, LBrace, RBrace, LBracket, RBracket, + Comma, Semi, Colon, ColonColon, Eq, EqEq, NotEq, Lt, LtEq, Gt, GtEq, + Pipe, Plus, Minus, Eof, +} + +fn tokenize(input: &str) -> Vec { + let mut tokens = Vec::new(); + let mut chars = input.chars().peekable(); + while let Some(&ch) = chars.peek() { + match ch { + ' ' | '\t' | '\n' | '\r' => { chars.next(); }, + '-' if chars.clone().nth(1).map_or(false, |c| c == '-') => { + while let Some(&c) = chars.peek() { chars.next(); if c == '\n' { break; } } + }, + '(' => { chars.next(); tokens.push(Token::LParen); }, + ')' => { chars.next(); tokens.push(Token::RParen); }, + '{' => { chars.next(); tokens.push(Token::LBrace); }, + '}' => { chars.next(); tokens.push(Token::RBrace); }, + '[' => { chars.next(); tokens.push(Token::LBracket); }, + ']' => { chars.next(); tokens.push(Token::RBracket); }, + ',' => { chars.next(); tokens.push(Token::Comma); }, + ';' => { chars.next(); tokens.push(Token::Semi); }, + '|' => { chars.next(); tokens.push(Token::Pipe); }, + '+' => { chars.next(); tokens.push(Token::Plus); }, + '=' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::EqEq); } else { tokens.push(Token::Eq); } }, + '!' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::NotEq); } else { panic!("Expected != after !"); } }, + '<' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::LtEq); } else { tokens.push(Token::Lt); } }, + '>' => { chars.next(); if chars.peek() == Some(&'=') { chars.next(); tokens.push(Token::GtEq); } else { tokens.push(Token::Gt); } }, + '$' => { chars.next(); tokens.push(Token::Dollar); }, + '-' => { chars.next(); tokens.push(Token::Minus); }, + ':' => { chars.next(); if chars.peek() == Some(&':') { chars.next(); tokens.push(Token::ColonColon); } else { tokens.push(Token::Colon); } }, + c if c.is_ascii_digit() => { + let mut num = String::new(); + while let Some(&c) = chars.peek() { if c.is_ascii_digit() { num.push(c); chars.next(); } else { break; } } + tokens.push(Token::Int(num.parse().unwrap())); + }, + c if c.is_ascii_alphabetic() || c == '_' => { + let mut ident = String::new(); + while let Some(&c) = chars.peek() { if c.is_ascii_alphanumeric() || c == '_' { ident.push(c); chars.next(); } else { break; } } + tokens.push(match ident.as_str() { + "let" => Token::Let, "var" => Token::Var, "result" => Token::Result, + "input" => Token::Input, "key" => Token::Key, "map" => Token::Map, + "join" => Token::Join, "min" => Token::Min, "distinct" => Token::Distinct, + "count" => Token::Count, "arrange" => Token::Arrange, "negate" => Token::Negate, + "filter" => Token::Filter, "enter_at" => Token::EnterAt, "inspect" => Token::Inspect, + _ => Token::Ident(ident), + }); + }, + other => panic!("Unexpected character: {:?}", other), + } + } + tokens.push(Token::Eof); + tokens +} + +struct Parser { tokens: Vec, pos: usize } + +impl Parser { + fn new(tokens: Vec) -> Self { Parser { tokens, pos: 0 } } + fn peek(&self) -> &Token { &self.tokens[self.pos] } + fn next(&mut self) -> Token { let t = self.tokens[self.pos].clone(); self.pos += 1; t } + fn expect(&mut self, expected: &Token) { let t = self.next(); assert_eq!(&t, expected, "Expected {:?}, got {:?}", expected, t); } + + fn parse_program(&mut self) -> Vec { + let mut stmts = Vec::new(); + while *self.peek() != Token::Eof && *self.peek() != Token::RBrace { stmts.push(self.parse_stmt()); } + stmts + } + + fn parse_stmt(&mut self) -> Stmt { + match self.peek().clone() { + Token::Let => { self.next(); let n = self.parse_ident(); self.expect(&Token::Eq); let e = self.parse_pipe_expr(); self.expect(&Token::Semi); Stmt::Let(n, e) }, + Token::Var => { self.next(); let n = self.parse_ident(); self.expect(&Token::Eq); let e = self.parse_pipe_expr(); self.expect(&Token::Semi); Stmt::Var(n, e) }, + Token::Result => { self.next(); let e = self.parse_pipe_expr(); self.expect(&Token::Semi); Stmt::Result(e) }, + Token::Ident(_) => { + let n = self.parse_ident(); self.expect(&Token::Colon); + self.expect(&Token::LBrace); let b = self.parse_program(); self.expect(&Token::RBrace); Stmt::Scope(n, b) + }, + other => panic!("Unexpected token: {:?}", other), + } + } + + fn parse_ident(&mut self) -> String { match self.next() { Token::Ident(s) => s, other => panic!("Expected ident, got {:?}", other) } } + + fn parse_pipe_expr(&mut self) -> Expr { + let mut expr = self.parse_concat_expr(); + while *self.peek() == Token::Pipe { + self.next(); + expr = self.parse_pipe_op(expr); + } + expr + } + + fn parse_concat_expr(&mut self) -> Expr { + let first = self.parse_atom(); + let mut parts = vec![first]; + loop { + match self.peek() { + Token::Plus => { self.next(); parts.push(self.parse_atom()); }, + Token::Minus => { self.next(); parts.push(Expr::Negate(Box::new(self.parse_atom()))); }, + _ => break, + } + } + if parts.len() == 1 { parts.pop().unwrap() } else { Expr::Concat(parts) } + } + + fn parse_atom(&mut self) -> Expr { + match self.peek().clone() { + Token::Input => { self.next(); match self.next() { Token::Int(n) => Expr::Input(n as usize), o => panic!("Expected int, got {:?}", o) } }, + Token::Ident(_) => { let n = self.parse_ident(); if *self.peek() == Token::ColonColon { self.next(); let f = self.parse_ident(); Expr::Qualified(n, f) } else { Expr::Name(n) } }, + Token::LParen => { self.next(); let e = self.parse_pipe_expr(); self.expect(&Token::RParen); e }, + other => panic!("Unexpected token in atom: {:?}", other), + } + } + + fn parse_join_arg(&mut self) -> Expr { + let mut expr = self.parse_atom(); + while *self.peek() == Token::Pipe { self.next(); expr = self.parse_pipe_op(expr); } + expr + } + + fn parse_pipe_op(&mut self, lhs: Expr) -> Expr { + match self.peek().clone() { + Token::Key => { self.next(); let p = self.parse_projection(); Expr::Map(Box::new(lhs), p) }, + Token::Map => { self.next(); let p = self.parse_projection(); Expr::Map(Box::new(lhs), p) }, + Token::Join => { self.next(); self.expect(&Token::LParen); let r = self.parse_join_arg(); self.expect(&Token::Comma); let p = self.parse_projection(); self.expect(&Token::RParen); Expr::Join(Box::new(lhs), Box::new(r), p) }, + Token::Min => { self.next(); Expr::Reduce(Box::new(lhs), Reducer::Min) }, + Token::Distinct => { self.next(); Expr::Reduce(Box::new(lhs), Reducer::Distinct) }, + Token::Count => { self.next(); Expr::Reduce(Box::new(lhs), Reducer::Count) }, + Token::Arrange => { self.next(); Expr::Arrange(Box::new(lhs)) }, + Token::Negate => { self.next(); Expr::Negate(Box::new(lhs)) }, + Token::Filter => { self.next(); self.expect(&Token::LParen); let c = self.parse_condition(); self.expect(&Token::RParen); Expr::Filter(Box::new(lhs), c) }, + Token::EnterAt => { self.next(); self.expect(&Token::LParen); let f = self.parse_field(); self.expect(&Token::RParen); Expr::EnterAt(Box::new(lhs), f) }, + Token::Inspect => { self.next(); self.expect(&Token::LParen); let l = self.parse_ident(); self.expect(&Token::RParen); Expr::Inspect(Box::new(lhs), l) }, + other => panic!("Expected pipe operator, got {:?}", other), + } + } + + fn parse_projection(&mut self) -> Projection { + self.expect(&Token::LParen); + self.parse_projection_inner() + } + + fn parse_projection_inner(&mut self) -> Projection { + if *self.peek() == Token::RParen { self.next(); return Projection { key: vec![], val: vec![] }; } + if *self.peek() == Token::Semi { self.next(); + if *self.peek() == Token::RParen { self.next(); return Projection { key: vec![], val: vec![] }; } + let mut val = vec![self.parse_field()]; + while *self.peek() == Token::Comma { self.next(); val.push(self.parse_field()); } + self.expect(&Token::RParen); + return Projection { key: vec![], val }; + } + let mut key = vec![self.parse_field()]; + while *self.peek() == Token::Comma { self.next(); key.push(self.parse_field()); } + let val = if *self.peek() == Token::Semi { self.next(); + if *self.peek() == Token::RParen { vec![] } + else { let mut v = vec![self.parse_field()]; while *self.peek() == Token::Comma { self.next(); v.push(self.parse_field()); } v } + } else { vec![] }; + self.expect(&Token::RParen); + Projection { key, val } + } + + fn parse_field(&mut self) -> FieldExpr { + match self.peek().clone() { + Token::Dollar => { self.next(); let n = match self.next() { Token::Int(n) => n as usize, o => panic!("Expected int, got {:?}", o) }; if *self.peek() == Token::LBracket { self.next(); let i = match self.next() { Token::Int(i) => i as usize, o => panic!("Expected int, got {:?}", o) }; self.expect(&Token::RBracket); FieldExpr::Index(n, i) } else { FieldExpr::Pos(n) } }, + Token::Minus => { self.next(); FieldExpr::Neg(Box::new(self.parse_field())) }, + Token::Int(n) => { self.next(); FieldExpr::Const(n) }, + other => panic!("Unexpected token in field: {:?}", other), + } + } + + fn parse_condition(&mut self) -> Condition { + let l = self.parse_field(); + let op = self.next(); + let r = self.parse_field(); + match op { + Token::EqEq => Condition::Eq(l, r), + Token::NotEq => Condition::Ne(l, r), + Token::Lt => Condition::Lt(l, r), + Token::LtEq => Condition::Le(l, r), + Token::Gt => Condition::Gt(l, r), + Token::GtEq => Condition::Ge(l, r), + o => panic!("Expected comparison operator, got {:?}", o), + } + } +} + +pub fn parse(input: &str) -> Vec { let tokens = tokenize(input); let mut p = Parser::new(tokens); p.parse_program() } diff --git a/interactive/src/plan/concat.rs b/interactive/src/plan/concat.rs deleted file mode 100644 index 2190b0359..000000000 --- a/interactive/src/plan/concat.rs +++ /dev/null @@ -1,40 +0,0 @@ -//! Concat expression plan. - -use std::hash::Hash; - -use timely::dataflow::Scope; - -use differential_dataflow::{Collection, ExchangeData}; -use plan::{Plan, Render}; -use {TraceManager, Time, Diff, Datum}; - -/// Merges the source collections. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct Concat { - /// Plan for the data source. - pub plans: Vec>, -} - -impl Render for Concat { - - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - arrangements: &mut TraceManager) -> VecCollection<'scope, Time, Vec, Diff> - { - use timely::dataflow::operators::Concatenate; - use differential_dataflow::AsCollection; - - let collections = - self.plans - .iter() - .map(|plan| plan.render(scope, arrangements).inner) - .collect::>(); - - scope - .concatenate(collections) - .as_collection() - } -} diff --git a/interactive/src/plan/filter.rs b/interactive/src/plan/filter.rs deleted file mode 100644 index 1de5fb3ae..000000000 --- a/interactive/src/plan/filter.rs +++ /dev/null @@ -1,99 +0,0 @@ -//! Predicate expression plan. - -use std::hash::Hash; -use serde::{Deserialize, Serialize}; -use timely::dataflow::Scope; - -use differential_dataflow::{VecCollection, ExchangeData}; -use crate::plan::{Plan, Render}; -use crate::{TraceManager, Time, Diff, Datum}; - -/// What to compare against. -/// -/// A second argument is either a constant or the index of another value. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub enum SecondArgument { - /// A constant value. - Constant(Value), - /// An index of another value. - Position(usize), -} - -impl SecondArgument { - /// Produces the indicated value. - pub fn value<'a>(&'a self, values: &'a [Value]) -> &'a Value { - match self { - SecondArgument::Constant(value) => value, - SecondArgument::Position(index) => &values[*index], - } - } -} - -/// Possible predicates to apply. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub enum Predicate { - /// Strictly less than. - LessThan(usize, SecondArgument), - /// Less than or equal. - LessEqual(usize, SecondArgument), - /// Strictly greater than. - GreaterThan(usize, SecondArgument), - /// Greater than or equal. - GreaterEqual(usize, SecondArgument), - /// Equal. - Equal(usize, SecondArgument), - /// Not equal. - NotEqual(usize, SecondArgument), - /// Any of a list of predicates. - Any(Vec>), - /// All of a list of predicates. - All(Vec>), - /// The complement of a predicate. - Not(Box>), -} - -impl Predicate { - /// Indicates if the predicate is satisfied. - pub fn satisfied(&self, values: &[Value]) -> bool { - match self { - Predicate::LessThan(index, other) => values[*index].lt(other.value(values)), - Predicate::LessEqual(index, other) => values[*index].le(other.value(values)), - Predicate::GreaterThan(index, other) => values[*index].gt(other.value(values)), - Predicate::GreaterEqual(index, other) => values[*index].ge(other.value(values)), - Predicate::Equal(index, other) => values[*index].eq(other.value(values)), - Predicate::NotEqual(index, other) => values[*index].ne(other.value(values)), - Predicate::Any(predicates) => predicates.iter().any(|p| p.satisfied(values)), - Predicate::All(predicates) => predicates.iter().all(|p| p.satisfied(values)), - Predicate::Not(predicate) => !predicate.satisfied(values), - } - } -} - -/// A plan stage filtering source tuples by the specified -/// predicate. Frontends are responsible for ensuring that the source -/// binds the argument symbols. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct Filter { - /// Logical predicate to apply. - pub predicate: Predicate, - /// Plan for the data source. - pub plan: Box>, -} - -impl Render for Filter { - - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff> - { - let predicate = self.predicate.clone(); - self.plan - .render(scope, collections, arrangements) - .filter(move |tuple| predicate.satisfied(tuple)) - } -} diff --git a/interactive/src/plan/join.rs b/interactive/src/plan/join.rs deleted file mode 100644 index f653184a5..000000000 --- a/interactive/src/plan/join.rs +++ /dev/null @@ -1,107 +0,0 @@ -//! Equijoin expression plan. - -use std::hash::Hash; -use serde::{Deserialize, Serialize}; - -use timely::dataflow::Scope; - -use differential_dataflow::{VecCollection, ExchangeData}; -use crate::plan::{Plan, Render}; -use crate::{TraceManager, Time, Diff, Datum}; - -/// A plan stage joining two source relations on the specified -/// symbols. Throws if any of the join symbols isn't bound by both -/// sources. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct Join { - /// Pairs of indices whose values must be equal. - pub keys: Vec<(usize, usize)>, - /// Plan for the left input. - pub plan1: Box>, - /// Plan for the right input. - pub plan2: Box>, -} - -impl Render for Join { - - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff> - { - // acquire arrangements for each input. - let keys1 = self.keys.iter().map(|key| key.0).collect::>(); - let mut trace1 = - if let Some(arrangement) = arrangements.get_keyed(&self.plan1, &keys1[..]) { - arrangement - } - else { - let keys = keys1.clone(); - let arrangement = - self.plan1 - .render(scope, collections, arrangements) - .map(move |tuple| - ( - // TODO: Re-use `tuple` for values. - keys.iter().map(|index| tuple[*index].clone()).collect::>(), - tuple - .into_iter() - .enumerate() - .filter(|(index,_value)| !keys.contains(index)) - .map(|(_index,value)| value) - .collect::>(), - ) - ) - .arrange_by_key(); - - arrangements.set_keyed(&self.plan1, &keys1[..], &arrangement.trace); - arrangement.trace - }; - - // extract relevant fields for each index. - let keys2 = self.keys.iter().map(|key| key.1).collect::>(); - let mut trace2 = - if let Some(arrangement) = arrangements.get_keyed(&self.plan2, &keys2[..]) { - arrangement - } - else { - let keys = keys2.clone(); - let arrangement = - self.plan2 - .render(scope, collections, arrangements) - .map(move |tuple| - ( - // TODO: Re-use `tuple` for values. - keys.iter().map(|index| tuple[*index].clone()).collect::>(), - tuple - .into_iter() - .enumerate() - .filter(|(index,_value)| !keys.contains(index)) - .map(|(_index,value)| value) - .collect::>(), - ) - ) - .arrange_by_key(); - - arrangements.set_keyed(&self.plan2, &keys2[..], &arrangement.trace); - arrangement.trace - }; - - let arrange1 = trace1.import(scope); - let arrange2 = trace2.import(scope); - - arrange1 - .join_core(arrange2, |keys, vals1, vals2| { - Some( - keys.iter().cloned() - .chain(vals1.iter().cloned()) - .chain(vals2.iter().cloned()) - .collect() - ) - }) - } -} diff --git a/interactive/src/plan/map.rs b/interactive/src/plan/map.rs deleted file mode 100644 index 728e99439..000000000 --- a/interactive/src/plan/map.rs +++ /dev/null @@ -1,47 +0,0 @@ -//! Projection expression plan. - -use std::hash::Hash; -use serde::{Deserialize, Serialize}; - -use timely::dataflow::Scope; - -use differential_dataflow::{VecCollection, ExchangeData}; -use crate::plan::{Plan, Render}; -use crate::{TraceManager, Time, Diff, Datum}; - -/// A plan which retains values at specified locations. -/// -/// The plan does not ascribe meaning to specific locations (e.g. bindings) -/// to variable names, and simply selects out the indicated sequence of values, -/// panicking if some input record is insufficiently long. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct Map { - /// Sequence (and order) of indices to be retained. - pub expressions: Vec, - /// Plan for the data source. - pub plan: Box>, -} - -impl Render for Map { - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff> - { - let expressions = self.expressions.clone(); - - // TODO: re-use `tuple` allocation. - self.plan - .render(scope, collections, arrangements) - .map(move |tuple| - expressions - .iter() - .map(|expr| V::subject_to(&tuple[..], expr)) - .collect() - ) - } -} diff --git a/interactive/src/plan/mod.rs b/interactive/src/plan/mod.rs deleted file mode 100644 index 7da44c926..000000000 --- a/interactive/src/plan/mod.rs +++ /dev/null @@ -1,229 +0,0 @@ -//! Types and traits for implementing query plans. - -use std::hash::Hash; -use serde::{Deserialize, Serialize}; - -use timely::dataflow::Scope; -use differential_dataflow::{VecCollection, ExchangeData}; - -use crate::{TraceManager, Time, Diff}; - -// pub mod count; -pub mod filter; -pub mod join; -pub mod map; -pub mod sfw; - -use crate::Datum; - -// pub use self::count::Count; -pub use self::filter::{Filter, Predicate}; -pub use self::join::Join; -pub use self::sfw::MultiwayJoin; -pub use self::map::Map; - -/// A type that can be rendered as a collection. -pub trait Render : Sized { - - /// Value type produced. - type Value: ExchangeData+Datum; - - /// Renders the instance as a collection in the supplied scope. - /// - /// This method has access to arranged data, and may rely on and update the set - /// of arrangements based on the needs and offerings of the rendering process. - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff>;} - -/// Possible query plan types. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub enum Plan { - /// Map - Map(Map), - /// Distinct - Distinct(Box>), - /// Concat - Concat(Vec>), - /// Consolidate - Consolidate(Box>), - /// Equijoin - Join(Join), - /// MultiwayJoin - MultiwayJoin(MultiwayJoin), - /// Negation - Negate(Box>), - /// Filters bindings by one of the built-in predicates - Filter(Filter), - /// Sources data from another relation. - Source(String), - /// Prints resulting updates. - Inspect(String, Box>), -} - -impl Plan { - /// Retains only the values at the indicated indices. - pub fn project(self, indices: Vec) -> Self { - Plan::Map(Map { - expressions: indices.into_iter().map(|i| V::projection(i)).collect(), - plan: Box::new(self), - }) - } - /// Reduces a collection to distinct tuples. - pub fn distinct(self) -> Self { - Plan::Distinct(Box::new(self)) - } - /// Merges two collections. - pub fn concat(self, other: Self) -> Self { - Plan::Concat(vec![self, other]) - } - /// Merges multiple collections. - pub fn concatenate(plans: Vec) -> Self { - Plan::Concat(plans) - } - /// Merges multiple collections. - pub fn consolidate(self) -> Self { - Plan::Consolidate(Box::new(self)) - } - /// Equi-joins two collections using the specified pairs of keys. - pub fn join(self, other: Plan, keys: Vec<(usize, usize)>) -> Self { - Plan::Join(Join { - keys, - plan1: Box::new(self), - plan2: Box::new(other), - }) - } - /// Equi-joins multiple collections using lists of equality constraints. - /// - /// The list `equalities` should contain equivalence classes of pairs of - /// attribute index and source index, and the `multiway_join` method will - /// ensure that each equivalence class has equal values in each attribute. - pub fn multiway_join( - sources: Vec, - equalities: Vec>, - results: Vec<(usize, usize)> - ) -> Self { - Plan::MultiwayJoin(MultiwayJoin { - results, - sources, - equalities, - }) - } - /// Negates a collection (negating multiplicities). - pub fn negate(self) -> Self { - Plan::Negate(Box::new(self)) - } - /// Restricts collection to tuples satisfying the predicate. - pub fn filter(self, predicate: Predicate) -> Self { - Plan::Filter(Filter { predicate, plan: Box::new(self) } ) - } - /// Loads a source of data by name. - pub fn source(name: &str) -> Self { - Plan::Source(name.to_string()) - } - /// Prints each tuple prefixed by `text`. - pub fn inspect(self, text: &str) -> Self { - Plan::Inspect(text.to_string(), Box::new(self)) - } - /// Convert the plan into a named rule. - pub fn into_rule(self, name: &str) -> crate::Rule { - crate::Rule { - name: name.to_string(), - plan: self, - } - } -} - -impl Render for Plan { - - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff> - { - if collections.get(self).is_none() { - - let collection = - match self { - // Plan::Project(projection) => projection.render(scope, collections, arrangements), - Plan::Map(expressions) => expressions.render(scope, collections, arrangements), - Plan::Distinct(distinct) => { - - use differential_dataflow::trace::implementations::{KeyBuilder, KeySpine}; - - let input = - if let Some(mut trace) = arrangements.get_unkeyed(&self) { - trace.import(scope) - } - else { - let input_arrangement = distinct.render(scope, collections, arrangements).arrange_by_self(); - arrangements.set_unkeyed(&distinct, &input_arrangement.trace); - input_arrangement - }; - - let output = input.reduce_abelian::<_,KeyBuilder<_,_,_>,KeySpine<_,_,_>,_>( - "Distinct", - move |_,_,t| t.push(((), 1)), - |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, - ); - - arrangements.set_unkeyed(&self, &output.trace); - output.as_collection(|k,&()| k.clone()) - - }, - Plan::Concat(concat) => { - - use timely::dataflow::operators::Concatenate; - use differential_dataflow::AsCollection; - - let plans = - concat - .iter() - .map(|plan| plan.render(scope, collections, arrangements).inner) - .collect::>(); - - scope - .concatenate(plans) - .as_collection() - } - Plan::Consolidate(consolidate) => { - if let Some(mut trace) = arrangements.get_unkeyed(&self) { - trace.import(scope).as_collection(|k,&()| k.clone()) - } - else { - consolidate.render(scope, collections, arrangements).consolidate() - } - }, - Plan::Join(join) => join.render(scope, collections, arrangements), - Plan::MultiwayJoin(join) => join.render(scope, collections, arrangements), - Plan::Negate(negate) => { - negate.render(scope, collections, arrangements).negate() - }, - Plan::Filter(filter) => filter.render(scope, collections, arrangements), - Plan::Source(source) => { - arrangements - .get_unkeyed(self) - .expect(&format!("Failed to find source collection: {:?}", source)) - .import(scope) - .as_collection(|k,()| k.to_vec()) - }, - Plan::Inspect(text, plan) => { - let text = text.clone(); - plan.render(scope, collections, arrangements) - .inspect(move |x| println!("{}\t{:?}", text, x)) - }, - }; - - collections.insert(self.clone(), collection); - } - - collections.get(self).expect("We just installed this").clone() - } -} diff --git a/interactive/src/plan/sfw.rs b/interactive/src/plan/sfw.rs deleted file mode 100644 index 81896ff95..000000000 --- a/interactive/src/plan/sfw.rs +++ /dev/null @@ -1,361 +0,0 @@ -//! Multi-way equijoin expression plan. -//! -//! This plan provides us the opportunity to map out a non-trivial differential -//! implementation for a complex join query. In particular, we are able to invoke -//! delta-query and worst-case optimal join plans, which avoid any intermediate -//! materialization. -//! -//! Each `MultiwayJoin` indicates several source collections, equality constraints -//! among their attributes, and then the set of attributes to produce as results. -//! -//! One naive implementation would take each input collection in order, and develop -//! the join restricted to the prefix of relations so far. Ideally the order would -//! be such that joined collections have equality constraints and prevent Cartesian -//! explosion. At each step, a new collection picks out some of the attributes and -//! instantiates a primitive binary join between the accumulated collection and the -//! next collection. -//! -//! A more sophisticated implementation establishes delta queries for each input -//! collection, which responds to changes in that input collection against the -//! current other input collections. For each input collection we may choose very -//! different join orders, as the order must follow equality constraints. -//! -//! A further implementation could develop the results attribute-by-attribute, as -//! opposed to collection-by-collection, which gives us the ability to use column -//! indices rather than whole-collection indices. - -use std::hash::Hash; -use serde::{Deserialize, Serialize}; - -use timely::dataflow::Scope; - -use differential_dataflow::{VecCollection, ExchangeData}; -use crate::plan::{Plan, Render}; -use crate::{TraceManager, Time, Diff, Datum}; - -/// A multiway join of multiple relations. -/// -/// By expressing multiple relations and required equivalances between their attributes, -/// we can more efficiently design incremental update strategies without materializing -/// and indexing intermediate relations. -#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct MultiwayJoin { - /// A list of (attribute index, input) pairs to extract. - pub results: Vec<(usize, usize)>, - /// A list of source collections. - pub sources: Vec>, - /// Equality constraints. - /// - /// Equality constraints are presented as lists of `(attr, input)` equivalence classes. - /// This means that each `(attr, input)` pair can exist in at most one list; if it would - /// appear in more than one list, those two lists should be merged. - pub equalities: Vec>, -} - -// TODO: This logic fails to perform restrictions in cases where a join does not -// occur. One example could be: -// -// result(a,b,c) := R1(a,b), R2(b,c,c) -// -// In this case, the requirement that the 2nd and 3rd columns of R2 be equal -// is not surfaced in any join, and is instead a filter that should be applied -// directly to R2 (before or after the join with R1; either could be best). - -impl Render for MultiwayJoin { - - type Value = V; - - fn render<'scope>( - &self, - scope: &mut Scope<'scope, Time>, - collections: &mut std::collections::HashMap, VecCollection<'scope, Time, Vec, Diff>>, - arrangements: &mut TraceManager, - ) -> VecCollection<'scope, Time, Vec, Diff> - { - // The idea here is the following: - // - // For each stream, we will determine a streaming delta query, in which changes - // are joined against indexed forms of the other relations using `dogsdogsdogs` - // stateless `propose` operators. - // - // For a query Q(x,y,z) := A(x,y), B(y,z), C(x,z) we might write dataflows like: - // - // dQdA := dA(x,y), B(y,z), C(x,z) - // dQdB := dB(y,z), A(x,y), C(x,z) - // dQdC := dC(x,y), A(x,y), B(y,z) - // - // where each line is read from left to right as a sequence of `propose` joins, - // which respond to timestamped delta changes by joining with the maintained - // relation at that corresponding time. - // - // We take some care to make sure that when these joins are performed, a delta - // interacts with relations as if we updated A, then B, then C, as if in sequence. - // That is, when a dB delta joins A it observes all updates to A at times less or - // equal to the delta's timestamp, but when a dB delta joins C it observes only - // updates to C at times strictly less than the delta's timestamp. - // - // This is done to avoid double counting updates; any concurrent changes will be - // accounted for by the last relation for which there is a concurrent update. - - // println!("{:?}", self); - - // Attributes we may need from any and all relations. - let mut relevant_attributes = Vec::new(); - relevant_attributes.extend(self.results.iter().cloned()); - relevant_attributes.extend(self.equalities.iter().flat_map(|list| list.iter().cloned())); - relevant_attributes.sort(); - relevant_attributes.dedup(); - - // println!("Relevant attributes: {:?}", relevant_attributes); - - // Into which we accumulate change streams. - let mut accumulated_changes = Vec::new(); - - // For each participating relation, we build a delta query dataflow. - for (index, plan) in self.sources.iter().enumerate() { - - // println!("building dataflow for relation {}", index); - - // Restrict down to relevant attributes. - let mut attributes: Vec<(usize, usize)> = - relevant_attributes - .iter() - .filter(|(_attr, input)| input == &index) - .cloned() - .collect::>(); - - let attributes_init = attributes.clone(); - // println!("\tinitial attributes: {:?}", attributes); - - // Ensure the plan is rendered and cached. - if arrangements.get_unkeyed(&plan).is_none() { - // println!("\tbuilding/caching source plan"); - let collection = plan.render(scope, collections, arrangements); - arrangements.set_unkeyed(plan, &collection.arrange_by_self().trace); - } - else { - // println!("\tsource plan found"); - } - let changes = - arrangements - .get_unkeyed(&plan) - .expect("Surely we just ensured this") - .import(scope) - .as_collection(|val,&()| val.clone()) - .map(move |tuple| attributes_init.iter().map(|&(attr,_)| - tuple[attr].clone()).collect::>() - ); - - // Before constructing the dataflow, which takes a borrow on `scope`, - // we'll want to ensure that we have all of the necessary data assets - // in place. This requires a bit of planning first, then the building. - - // Acquire a sane sequence in which to join the relations: - // - // This is a sequence of relation identifiers, starting with `index`, - // such that each has at least one attribute in common with a prior - // relation, and so can be effectively joined. - let join_order = plan_join_order(index, &self.equalities); - let mut join_plan = Vec::new(); - - // println!("\tjoin order: {:?}", join_order); - - // Skipping `index`, join in each relation in sequence. - for join_idx in join_order.into_iter().skip(1) { - - // To join a relation, we need to determine any constraints on - // attributes in common with prior relations. Any other values - // should be appended to tuples in `changes` with care taken to - // update `attributes`. - let (keys, priors) = determine_keys_priors(join_idx, &self.equalities, &attributes[..]); - - // The fields in `sources[join_idx]` that should be values are those - // that are required output or participate in an equality constraint, - // but *WHICH ARE NOT* in `keys`. - let vals = - relevant_attributes - .iter() - .filter(|&(attr,index)| index == &join_idx && !keys.contains(&attr)) - .cloned() - .collect::>(); - - // println!("\tkeys: {:?}, priors: {:?}, vals: {:?}", keys, priors, vals); - - let mut projection = Vec::new(); - for &attr in keys.iter() { - projection.push(attr); - } - for &(attr, _index) in vals.iter() { - projection.push(attr); - } - // TODO: Sort, to improve chances of re-use opportunities. - // Requires understanding how attributes move to get the right - // key selectors out though. - // projection.sort(); - // projection.dedup(); // Should already be deduplicated, probably? - - // Get a plan for the projection on to these few attributes. - let plan = self.sources[join_idx].clone().project(projection); - - if arrangements.get_keyed(&plan, &keys[..]).is_none() { - // println!("\tbuilding key: {:?}, plan: {:?}", keys, plan); - let keys_clone = keys.clone(); - let arrangement = - plan.render(scope, collections, arrangements) - .map(move |tuple| (keys_clone.iter().map(|&i| tuple[i].clone()).collect::>(), tuple)) - .arrange_by_key(); - - arrangements.set_keyed(&plan, &keys[..], &arrangement.trace); - } - else { - // println!("\tplan found: {:?}, {:?}", keys, plan); - } - - let arrangement = - arrangements - .get_keyed(&plan, &keys[..]) - .expect("Surely we just ensured this"); - - let key_selector = move |change: &Vec| - priors.iter().map(|&p| change[p].clone()).collect::>() - ; - - join_plan.push((join_idx, key_selector, arrangement)); - - attributes.extend(keys.into_iter().map(|x| (x, join_idx))); - attributes.extend(vals.into_iter()); - // println!("\tattributes: {:?}", attributes); - } - - // Build the dataflow. - use differential_dogs3::altneu::AltNeu; - - let scope_name = format!("DeltaRule: {}/{}", index, self.sources.len()); - let changes = scope.scoped::,_,_>(&scope_name, |inner| { - - // This should default to an `AltNeu::Alt` timestamp. - let mut changes = - changes - .enter(inner) - ; - - for (join_idx, key_selector, mut trace) in join_plan.into_iter() { - - // Use alt or neu timestamps based on relative indices. - // Must have an `if` statement here as the two arrangement have different - // types, and we would to determine `alt` v `neu` once, rather than per - // tuple in the cursor. - changes = - if join_idx < index { - let arrangement = trace.import(scope).enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |_| unimplemented!()); - differential_dogs3::operators::propose(changes, arrangement, key_selector) - } - else { - let arrangement = trace.import(scope).enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |_| unimplemented!()); - differential_dogs3::operators::propose(changes, arrangement, key_selector) - } - .map(|(mut prefix, extensions)| { prefix.extend(extensions.into_iter()); prefix }) - ; - - // TODO: Equality constraints strictly within a relation have the effect - // of "filtering" data, but they are ignored at the moment. We should - // check for these and do something about it. - } - - // Extract `self.results` in order, using `attributes`. - // - // The specific attribute requested in `self.results` may not be present in - // `attributes` when it is equal to another present attribute. So, we should - // look around in `self.equalities` also. - let mut extract_map = Vec::new(); - for result in self.results.iter() { - if let Some(position) = attributes.iter().position(|i| i == result) { - extract_map.push(position); - } - else { - for constraint in self.equalities.iter() { - if constraint.contains(result) { - if let Some(position) = constraint.iter().flat_map(|x| attributes.iter().position(|i| i == x)).next() { - extract_map.push(position); - } - else { - println!("WTF NOTHING FOUND NOOOOO!!!"); - } - } - } - } - } - - changes - .map(move |tuple| extract_map.iter().map(|&i| tuple[i].clone()).collect::>()) - .leave(scope) - }); - - accumulated_changes.push(changes); - } - - differential_dataflow::collection::concatenate(scope, accumulated_changes.into_iter()) - .consolidate() - } -} - -/// Sequences relations in `constraints`. -/// -/// Relations become available for sequencing as soon as they share a constraint with -/// either `source` or another sequenced relation. -fn plan_join_order(source: usize, constraints: &[Vec<(usize, usize)>]) -> Vec { - - let mut result = vec![source]; - let mut active = true; - while active { - active = false; - for constraint in constraints.iter() { - // Check to see if the constraint contains a sequenced relation. - if constraint.iter().any(|(_,index)| result.contains(index)) { - // If so, sequence any unsequenced relations. - for (_, index) in constraint.iter() { - if !result.contains(index) { - result.push(*index); - active = true; - } - } - } - } - } - - result -} - -/// Identifies keys and values for a join. -/// -/// The result is a sequence, for each -fn determine_keys_priors( - relation: usize, - constraints: &[Vec<(usize, usize)>], - current_attributes: &[(usize, usize)], -) --> (Vec, Vec) -{ - // The fields in `sources[join_idx]` that should be keys are those - // that share an equality constraint with an element of `attributes`. - // For each key, we should capture the associated `attributes` entry - // so that we can easily prepare the keys of the `delta` stream. - let mut keys = Vec::new(); - let mut priors = Vec::new(); - for constraint in constraints.iter() { - - // If there is an intersection between `constraint` and `current_attributes`, - // we should capture the position in `current_attributes` and emit all of the - // attributes for `relation`. - if let Some(prior) = current_attributes.iter().position(|x| constraint.contains(x)) { - for &(attr, index) in constraint.iter() { - if index == relation { - keys.push(attr); - priors.push(prior); - } - } - } - } - - (keys, priors) -} From 6f2f19e3fea36b5b2622c9466eff7a4ad601b5bd Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Sat, 11 Apr 2026 18:17:57 -0400 Subject: [PATCH 24/27] DDIR README --- interactive/README.md | 99 +++++++++++++++++++++++++ interactive/examples/programs/reach.ddp | 13 ++++ 2 files changed, 112 insertions(+) create mode 100644 interactive/README.md create mode 100644 interactive/examples/programs/reach.ddp diff --git a/interactive/README.md b/interactive/README.md new file mode 100644 index 000000000..c72c07436 --- /dev/null +++ b/interactive/README.md @@ -0,0 +1,99 @@ +# DDIR + +An intermediate representation of differential dataflow suitable for interpretation. + +## Background + +Differential dataflow is a framework that supports declarative computation over continually changing data. +One of its primary irritants is its compile times, due to an over-indulgence in monomorphization. +This project looks at isolating an interpretable core, which lays the groundwork for experimentation in different concrete syntaxes, optimization, and back-ends. + +## An example: reachability + +Here's an example that performs graph reachability: + +``` +let edges = input 0 | key($0[0] ; $0[1]); +let roots = input 1 | key($0[0] ;); + +reach: { + let label = reach | join(edges, ($2 ;)); + var reach = roots + label | distinct; +} + +result reach::reach | key(;) | arrange | inspect(total); +``` + +Informally, this computation starts from graph `edges` and nodes `roots`, and repeatedly expands a set of nodes that can be reached from each root. +The details of this example are somewhat arbitrary, but the `let`, `var`, and `{ .. }` bits are quite important. +Everything else does have a role, but what they do and how they are expressed is intentionally fluid. +If you are familiar with differential dataflow, they correspond to various of its transformation operators. + +## Syntax and Semantics + +Programs in DDIR are structured as a tree of nested "iterative scopes". +Within each scope one can + +1. let-bind names to expressions, +2. name and bind iteration variables, +3. create further nested scopes. + +The expression language is on collections of data, and involves differential's relatively few operators: join, reduce, concat, and a surprisingly general flatmap operator. +The particular details of the expression language are evolving, and the intent is to let anyone pick their favorite concrete syntax. + +The intended semantics assign values to the names in each scope through an iterative process: + +1. Initially, each variable is the empty collection. +2. Repeatedly, all variables synchronously update to new values, as a function of their prior values. +3. Eventually, the value of each named variable is its fixed point under this process. + +References between scopes are resolved at their least common ancestor. +A reference sees the converged value as observed from the ancestor scope. + +## Design goals + +The IR is meant to be amenable to reasoning and manipulation, and has some properties that support this. + +1. **Declarative**: the statement order does not affect the semantics. +2. **Functional**: each value is determined from a pure function of its inputs. +3. **Unambiguous**: the name-to-value map is fixed within each iteration. +4. **Equational**: all statement equalities within a scope hold outside the scope. + +The language is meant to be referentially transparent, and support equational substitution. +I don't know much about designing languages, so I may have gotten this wrong. + +## Architecture + +The architecture is fairly standard, and where it isn't it probably should become more standard. +The flow moves through four steps: + +1. The `parse/` directory contains any number of concrete syntax parsers. +2. The `lower/` directory contains lowering from the AST to the IR. +3. The `ir/` directory is the IR itself, with optimizations. +4. The `examples/` directory contains back-ends that execute programs. + +The `examples/programs/` directory contains example programs, intentionally simple at the moment. +You can run any of them with one of the example harnesses, for example +``` +cargo run --release --example ddir_vec -- ./examples/programs/reach.ddp 2 100 200 1 100 +``` + +More generally, you can run +``` +cargo run --release --example ddir_vec -- [] +``` +where +* `` is a path to your program file, +* `` is the number of columns expected by your program, +* `` is the range of values from zero for each column, +* `` is the number of records the harness will maintain, +* `` is the number of records the harness will change in each round, +* `` is the number of rounds the harness will perform. + +You can leave off the rounds, or any suffix really, to watch it just run for a while. + +## Status + +This is a research project, primarily for personal learning at this point. +Various bits of what is written above are not yet entirely true. +I would not recommend relying on any of this yet. diff --git a/interactive/examples/programs/reach.ddp b/interactive/examples/programs/reach.ddp new file mode 100644 index 000000000..36d24b979 --- /dev/null +++ b/interactive/examples/programs/reach.ddp @@ -0,0 +1,13 @@ +-- Reachability — pipe syntax +-- input 0: edges (src, dst ;) +-- input 1: roots (node ;) + +let edges = input 0 | key($0[0] ; $0[1]); +let roots = input 1 | key($0[0] ;); + +reach: { + let proposals = reach | join(edges, ($2 ;)); + var reach = roots + proposals | distinct; +} + +result reach::reach | key(;) | arrange | inspect(total); From 56dc613610cb829b36dfcd25245becab780dec9b Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 13 Apr 2026 09:55:25 -0400 Subject: [PATCH 25/27] Continue to track timely master (#720) --- Cargo.toml | 2 +- README.md | 4 ++-- advent_of_code_2017/src/bin/day_06.rs | 4 ++-- advent_of_code_2017/src/bin/day_07.rs | 4 ++-- advent_of_code_2017/src/bin/day_08.rs | 2 +- advent_of_code_2017/src/bin/day_09.rs | 6 +++--- advent_of_code_2017/src/bin/day_12.rs | 4 ++-- diagnostics/src/logging.rs | 8 ++++---- differential-dataflow/examples/accumulate.rs | 2 +- differential-dataflow/examples/arrange.rs | 7 +++---- differential-dataflow/examples/bfs.rs | 4 ++-- .../examples/columnar/columnar_support.rs | 10 +++++----- .../examples/columnar/main.rs | 2 +- differential-dataflow/examples/dynamic.rs | 2 +- differential-dataflow/examples/graspan.rs | 6 +++--- .../examples/iterate_container.rs | 2 +- differential-dataflow/examples/monoid-bfs.rs | 2 +- differential-dataflow/examples/pagerank.rs | 2 +- differential-dataflow/examples/progress.rs | 10 +++++----- .../examples/stackoverflow.rs | 4 ++-- .../src/algorithms/graphs/bfs.rs | 4 ++-- .../src/algorithms/graphs/bijkstra.rs | 2 +- .../src/algorithms/graphs/propagate.rs | 6 +++--- .../src/algorithms/graphs/scc.rs | 6 +++--- .../src/algorithms/graphs/sequential.rs | 4 ++-- .../src/algorithms/identifiers.rs | 8 ++++---- .../src/algorithms/prefix_sum.rs | 6 +++--- differential-dataflow/src/capture.rs | 15 +++++++-------- differential-dataflow/src/collection.rs | 16 ++++++++-------- differential-dataflow/src/input.rs | 16 ++++++++-------- .../src/operators/arrange/agent.rs | 11 +++++------ .../src/operators/arrange/arrangement.rs | 14 ++++++-------- .../src/operators/arrange/upsert.rs | 6 ++---- differential-dataflow/src/operators/iterate.rs | 14 +++++++------- differential-dataflow/src/operators/join.rs | 1 - differential-dataflow/src/operators/reduce.rs | 6 ++---- differential-dataflow/tests/bfs.rs | 4 ++-- differential-dataflow/tests/scc.rs | 6 +++--- dogsdogsdogs/examples/delta_query.rs | 4 ++-- dogsdogsdogs/examples/delta_query_wcoj.rs | 2 +- dogsdogsdogs/src/calculus.rs | 8 ++++---- dogsdogsdogs/src/lib.rs | 6 +++--- dogsdogsdogs/src/operators/half_join.rs | 1 - dogsdogsdogs/src/operators/half_join2.rs | 1 - experiments/src/bin/attend.rs | 4 ++-- experiments/src/bin/deals-interactive.rs | 18 +++++++++--------- experiments/src/bin/deals.rs | 4 ++-- experiments/src/bin/graphs-interactive-alt.rs | 6 +++--- experiments/src/bin/graphs-interactive-neu.rs | 2 +- experiments/src/bin/graphs-interactive.rs | 2 +- experiments/src/bin/graphs-static.rs | 10 +++++----- experiments/src/bin/graphs.rs | 16 ++++++++-------- experiments/src/bin/graspan-interactive.rs | 4 ++-- experiments/src/bin/graspan1.rs | 2 +- experiments/src/bin/graspan2.rs | 4 ++-- experiments/src/bin/multitemporal.rs | 4 ++-- interactive/examples/ddir_col.rs | 2 +- interactive/examples/ddir_vec.rs | 2 +- mdbook/src/chapter_2/chapter_2_7.md | 4 ++-- mdbook/src/chapter_4/chapter_4_1.md | 4 ++-- mdbook/src/chapter_5/chapter_5_4.md | 4 ++-- server/dataflows/random_graph/src/lib.rs | 2 +- server/dataflows/reachability/src/lib.rs | 4 ++-- server/src/lib.rs | 2 +- tpchlike/src/bin/arrange.rs | 2 +- tpchlike/src/bin/just-arrange.rs | 2 +- 66 files changed, 173 insertions(+), 185 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 895c0407e..96ebd6d9d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,7 +25,7 @@ rust-version = "1.86" differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.22.0" } #timely = { version = "0.28", default-features = false } columnar = { version = "0.12", default-features = false } -timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } +timely = { git = "https://github.com/TimelyDataflow/timely-dataflow", default-features = false } #timely = { path = "../timely-dataflow/timely/", default-features = false } [workspace.lints.clippy] diff --git a/README.md b/README.md index 41cfe05f1..d6a1b3e93 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,7 @@ Alternately, here is a fragment that computes the set of nodes reachable from a ```rust let reachable = roots.iterate(|scope, reach| - edges.enter(&scope) + edges.enter(scope) .semijoin(reach) .map(|(src, dst)| dst) .concat(reach) @@ -337,7 +337,7 @@ edges.iterate(|scope, inner| { .map(|(node,_)| node); // keep edges between active vertices - edges.enter(&scope) + edges.enter(scope) .semijoin(active) .map(|(src,dst)| (dst,src)) .semijoin(active) diff --git a/advent_of_code_2017/src/bin/day_06.rs b/advent_of_code_2017/src/bin/day_06.rs index 5cae0bef8..10017e224 100644 --- a/advent_of_code_2017/src/bin/day_06.rs +++ b/advent_of_code_2017/src/bin/day_06.rs @@ -24,7 +24,7 @@ fn main() { let stable = banks.iterate(|scope, iter| iter.map_in_place(|banks| recycle(banks)) - .concat(banks.enter(&scope)) + .concat(banks.enter(scope)) .distinct() ); @@ -43,7 +43,7 @@ fn main() { loop_point .iterate(|scope, iter| iter.map_in_place(|banks| recycle(banks)) - .concat(loop_point.enter(&scope)) + .concat(loop_point.enter(scope)) .distinct() ) .map(|_| ((),())) diff --git a/advent_of_code_2017/src/bin/day_07.rs b/advent_of_code_2017/src/bin/day_07.rs index 13b35094e..0d4a91be2 100644 --- a/advent_of_code_2017/src/bin/day_07.rs +++ b/advent_of_code_2017/src/bin/day_07.rs @@ -1104,10 +1104,10 @@ tvhftq (35)"; let total_weights: VecCollection<_,String> = weights .iterate(|scope, inner| { - parents.enter(&scope) + parents.enter(scope) .semijoin(inner) .map(|(_, parent)| parent) - .concat(weights.enter(&scope)) + .concat(weights.enter(scope)) }); parents diff --git a/advent_of_code_2017/src/bin/day_08.rs b/advent_of_code_2017/src/bin/day_08.rs index 385bc6489..e55aacee3 100644 --- a/advent_of_code_2017/src/bin/day_08.rs +++ b/advent_of_code_2017/src/bin/day_08.rs @@ -1107,7 +1107,7 @@ wui inc -120 if i > -2038"; .map(|_| ((0, String::new()), 0)) .iterate(|scope, valid| { - let edits = edits.enter(&scope); + let edits = edits.enter(scope); valid .prefix_sum_at(edits.map(|(key,_)| key), 0, |_k,x,y| *x + *y) diff --git a/advent_of_code_2017/src/bin/day_09.rs b/advent_of_code_2017/src/bin/day_09.rs index b462993cf..8a9a0f310 100644 --- a/advent_of_code_2017/src/bin/day_09.rs +++ b/advent_of_code_2017/src/bin/day_09.rs @@ -120,7 +120,7 @@ where if input.len() > 1 { result = combine(result, &(input[1].0).1); } output.push((result, 1)); }) - .concat(unit_ranges.enter(&scope)) + .concat(unit_ranges.enter(scope)) ) } @@ -154,10 +154,10 @@ where .iterate(|scope, state| { aggregates .filter(|&((_, log),_)| log < 64) // the log = 64 interval doesn't help us here (overflows). - .enter(&scope) + .enter(scope) .map(|((pos, log), data)| (pos, (log, data))) .join_map(state, move |&pos, &(log, ref data), state| (pos + (1 << log), combine(state, data))) - .concat(init_state.enter(&scope)) + .concat(init_state.enter(scope)) .distinct() }) .consolidate() diff --git a/advent_of_code_2017/src/bin/day_12.rs b/advent_of_code_2017/src/bin/day_12.rs index 4c92ee174..238b7bd51 100644 --- a/advent_of_code_2017/src/bin/day_12.rs +++ b/advent_of_code_2017/src/bin/day_12.rs @@ -2035,8 +2035,8 @@ fn main() { let labels = nodes .iterate(|scope, label| { - let edges = edges.enter(&scope); - let nodes = nodes.enter(&scope); + let edges = edges.enter(scope); + let nodes = nodes.enter(scope); label .join_map(edges, |_src, &lbl, &tgt| (tgt, lbl)) .concat(nodes) diff --git a/diagnostics/src/logging.rs b/diagnostics/src/logging.rs index c16490865..cc4b8331a 100644 --- a/diagnostics/src/logging.rs +++ b/diagnostics/src/logging.rs @@ -415,7 +415,7 @@ struct TimelyDemuxState { /// Build timely logging collections and arrangements. fn construct_timely<'scope>( - scope: &mut Scope<'scope, Duration>, + scope: Scope<'scope, Duration>, stream: Stream<'scope, Duration, Vec<(Duration, TimelyEvent)>>, ) -> (TimelyTraces, TimelyCollections<'scope>) { type OpUpdate = ((usize, String, Vec), Duration, i64); @@ -423,7 +423,7 @@ fn construct_timely<'scope>( type ElUpdate = (usize, Duration, i64); type MsgUpdate = (usize, Duration, i64); - let mut demux = OperatorBuilder::new("Timely Demux".to_string(), scope.clone()); + let mut demux = OperatorBuilder::new("Timely Demux".to_string(), scope); let mut input = demux.new_input(stream, Pipeline); let (op_out, operates) = demux.new_output::>(); @@ -546,12 +546,12 @@ struct DifferentialCollections<'scope> { /// Build differential logging collections and arrangements. fn construct_differential<'scope>( - scope: &mut Scope<'scope, Duration>, + scope: Scope<'scope, Duration>, stream: Stream<'scope, Duration, Vec<(Duration, DifferentialEvent)>>, ) -> (DifferentialTraces, DifferentialCollections<'scope>) { type Update = (usize, Duration, i64); - let mut demux = OperatorBuilder::new("Differential Demux".to_string(), scope.clone()); + let mut demux = OperatorBuilder::new("Differential Demux".to_string(), scope); let mut input = demux.new_input(stream, Pipeline); let (bat_out, batches) = demux.new_output::>(); diff --git a/differential-dataflow/examples/accumulate.rs b/differential-dataflow/examples/accumulate.rs index d8da7bdcd..588290acc 100644 --- a/differential-dataflow/examples/accumulate.rs +++ b/differential-dataflow/examples/accumulate.rs @@ -20,7 +20,7 @@ fn main() { scope.iterative::(|inner| { data.enter_at(inner, |_| 0) .consolidate() - .leave(&scope) + .leave(scope) }); input diff --git a/differential-dataflow/examples/arrange.rs b/differential-dataflow/examples/arrange.rs index d3f5abcec..2e5623679 100644 --- a/differential-dataflow/examples/arrange.rs +++ b/differential-dataflow/examples/arrange.rs @@ -2,7 +2,6 @@ use rand::{Rng, SeedableRng, StdRng}; use timely::dataflow::operators::*; use timely::order::Product; -use timely::scheduling::Scheduler; use differential_dataflow::input::Input; use differential_dataflow::AsCollection; @@ -109,8 +108,8 @@ fn main() { // repeatedly update minimal distances each node can be reached from each root roots.clone().iterate(|scope, dists| { - let edges = edges.enter(&scope); - let roots = roots.enter(&scope); + let edges = edges.enter(scope); + let roots = roots.enter(scope); dists.arrange_by_key() .join_core(edges, |_k,l,d| Some((*d, l+1))) @@ -175,4 +174,4 @@ fn main() { } } }).unwrap(); -} \ No newline at end of file +} diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index f8abe4873..aa87b72bc 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -100,8 +100,8 @@ where // repeatedly update minimal distances each node can be reached from each root nodes.clone().iterate(|scope, inner| { - let nodes = nodes.enter(&scope); - let edges = edges.enter(&scope); + let nodes = nodes.enter(scope); + let edges = edges.enter(scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index 03a260218..1241702e7 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -291,7 +291,7 @@ mod distributor { use timely::dataflow::channels::Message; use timely::dataflow::channels::pact::{LogPuller, LogPusher, ParallelizationContract}; use timely::progress::Timestamp; - use timely::worker::AsWorker; + use timely::worker::Worker; use crate::layout::ColumnarUpdate as Update; use crate::{Updates, RecordedUpdates}; @@ -361,15 +361,15 @@ mod distributor { >; type Puller = LogPuller>>>>; - fn connect(self, allocator: &mut A, identifier: usize, address: Rc<[usize]>, logging: Option) -> (Self::Pusher, Self::Puller) { - let (senders, receiver) = allocator.allocate::>>(identifier, address); - let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, allocator.index(), i, identifier, logging.clone())).collect::>(); + fn connect(self, worker: &Worker, identifier: usize, address: Rc<[usize]>, logging: Option) -> (Self::Pusher, Self::Puller) { + let (senders, receiver) = worker.allocate::>>(identifier, address); + let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, worker.index(), i, identifier, logging.clone())).collect::>(); let distributor = ValDistributor { marker: std::marker::PhantomData, hashfunc: self.hashfunc, pre_lens: Vec::new(), }; - (Exchange::new(senders, distributor), LogPuller::new(receiver, allocator.index(), identifier, logging.clone())) + (Exchange::new(senders, distributor), LogPuller::new(receiver, worker.index(), identifier, logging.clone())) } } } diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs index 3ccaec945..56380089c 100644 --- a/differential-dataflow/examples/columnar/main.rs +++ b/differential-dataflow/examples/columnar/main.rs @@ -185,7 +185,7 @@ mod reachability { variable.set(result_col.clone()); // Leave the iterative scope. - result_col.leave(&outer) + result_col.leave(outer) }) } } diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index 89ca2b2ae..077662f48 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -126,7 +126,7 @@ where // Leave the dynamic iteration, stripping off the last timestamp coordinate. next.leave_dynamic(1) .inspect(|x| println!("{:?}", x)) - .leave(&outer) + .leave(outer) }) } diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index 2f4ae3763..7b6117d24 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -89,7 +89,7 @@ pub struct EdgeVariable<'scope, T: Timestamp + Lattice> { impl<'scope, T: Timestamp + Lattice> EdgeVariable<'scope, T> { /// Creates a new variable initialized with `source`. pub fn from(source: VecCollection<'scope, T, Edge>, step: T::Summary) -> Self { - let (variable, collection) = VecVariable::new(&mut source.scope(), step); + let (variable, collection) = VecVariable::new(source.scope(), step); EdgeVariable { variable, collection, @@ -150,7 +150,7 @@ impl Query { } /// Creates a dataflow implementing the query, and returns input and trace handles. - pub fn render_in(&self, scope: &mut Scope) -> BTreeMap> + pub fn render_in(&self, scope: Scope) -> BTreeMap> where T: Timestamp + Lattice + ::timely::order::TotalOrder, { @@ -170,7 +170,7 @@ impl Query { // create variables and result handles for each named relation. for (name, (input, collection)) in input_map { let edge_variable = EdgeVariable::from(collection.enter(subscope), Product::new(Default::default(), 1)); - let trace = edge_variable.collection.clone().leave(&scope).arrange_by_self().trace; + let trace = edge_variable.collection.clone().leave(scope).arrange_by_self().trace; result_map.insert(name.clone(), RelationHandles { input, trace }); variable_map.insert(name.clone(), edge_variable); } diff --git a/differential-dataflow/examples/iterate_container.rs b/differential-dataflow/examples/iterate_container.rs index d2fa85302..bd7492df0 100644 --- a/differential-dataflow/examples/iterate_container.rs +++ b/differential-dataflow/examples/iterate_container.rs @@ -77,7 +77,7 @@ fn main() { }).as_collection().consolidate(); let result = wrap(result.inner).as_collection(); variable.set(result); - collection.leave(&scope) + collection.leave(scope) }); }) } diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index 027384cd8..fc0022b5d 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -152,6 +152,6 @@ where .as_collection(|k,()| *k); variable.set(result.clone()); - result.leave(&outer) + result.leave(outer) }) } diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index b649316ad..08ab898a1 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -131,6 +131,6 @@ where // Bind the recursive variable, return its limit. ranks_bind.set(pushed.clone()); - pushed.leave(&outer) + pushed.leave(outer) }) } diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 20550b7ea..809e50628 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -131,10 +131,10 @@ where .clone() .iterate(|scope, reach| { transitions - .enter(&scope) + .enter(scope) .join_map(reach, |_from, (dest, summ), time| (dest.clone(), summ.results_in(time))) .flat_map(|(dest, time)| time.map(move |time| (dest, time))) - .concat(times.enter(&scope)) + .concat(times.enter(scope)) .reduce(|_location, input, output: &mut Vec<(T, isize)>| { // retain the lower envelope of times. for (t1, _count1) in input.iter() { @@ -173,10 +173,10 @@ where .clone() .iterate(|scope, summaries| { transitions - .enter(&scope) + .enter(scope) .join_map(summaries, |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2))) .flat_map(|(from, to, summ)| summ.map(move |summ| (from, (to, summ)))) - .concat(zero_inputs.enter(&scope)) + .concat(zero_inputs.enter(scope)) .map(|(from, (to, summary))| ((from, to), summary)) .reduce(|_from_to, input, output| { for (summary, _count) in input.iter() { @@ -222,7 +222,7 @@ where .map(|(_source, target)| target) .distinct(); transitions - .enter(&scope) + .enter(scope) .semijoin(active) }) .consolidate() diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index 234930932..7aec81f29 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -114,8 +114,8 @@ where // repeatedly update minimal distances each node can be reached from each root nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&scope); - let nodes = nodes.enter(&scope); + let edges = edges.enter(scope); + let nodes = nodes.enter(scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index e1756b073..830cd0f6e 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -33,8 +33,8 @@ where // repeatedly update minimal distances each node can be reached from each root nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&scope); - let nodes = nodes.enter(&scope); + let edges = edges.enter(scope); + let nodes = nodes.enter(scope); inner.join_core(edges, |_k,l,d| Some((d.clone(), l+1))) .concat(nodes) diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index f6dfa520e..b9d2dd44c 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -118,6 +118,6 @@ where reverse_bind.set(reverse_next); - reached.leave(&outer) + reached.leave(outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 033ec3e9f..915bf545d 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -69,8 +69,8 @@ where // nodes.filter(|_| false) // .iterate(|scope, inner| { - // let edges = edges.enter(&scope); - // let nodes = nodes.enter_at(&scope, move |r| 256 * (64 - (logic(&r.1)).leading_zeros() as u64)); + // let edges = edges.enter(scope); + // let nodes = nodes.enter_at(scope, move |r| 256 * (64 - (logic(&r.1)).leading_zeros() as u64)); // inner.join_map(edges, |_k,l,d| (d.clone(),l.clone())) // .concat(nodes) // .reduce(|_, s, t| t.push((s[0].0.clone(), 1))) @@ -104,6 +104,6 @@ where labels .as_collection(|k,v| (k.clone(), v.clone())) - .leave(&outer) + .leave(outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index d50a5a05d..7dcb1de7f 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -24,7 +24,7 @@ where let outer = graph.scope(); outer.scoped::,_,_>("StronglyConnected", |scope| { // Bring in edges and transposed edges. - let edges = graph.enter(&scope); + let edges = graph.enter(scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); // Create a new variable that will be intra-scc edges. use crate::operators::iterate::Variable; @@ -32,7 +32,7 @@ where let result = trim_edges(trim_edges(inner, edges), trans); variable.set(result.clone()); - result.leave(&outer) + result.leave(outer) }) } @@ -64,6 +64,6 @@ where .join_core(labels, |e2,(e1,l1),l2| [((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))]) .filter(|(_,(l1,l2))| l1 == l2) .map(|((x1,x2),_)| (x2,x1)) - .leave_region(&outer) + .leave_region(outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index e264ed421..9c58cac61 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -57,8 +57,8 @@ where .map(|(node, _state)| (node, None)) .iterate(|scope, new_state| { // immutable content: edges and initial state. - let edges = edges.enter(&scope); - let old_state = state.enter(&scope); + let edges = edges.enter(scope); + let old_state = state.enter(scope); // .map(|x| (x.0, Some(x.1))); // break edges into forward and reverse directions. diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index c348613e7..18292e607 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -55,11 +55,11 @@ where use crate::collection::AsCollection; let init = self.map(|record| (0, record)); - timely::dataflow::operators::generic::operator::empty(&init.scope()) + timely::dataflow::operators::generic::operator::empty(init.scope()) .as_collection() .iterate(|scope, diff| init.clone() - .enter(&scope) + .enter(scope) .concat(diff) .map(|pair| (pair.hashed(), pair)) .reduce(|_hash, input, output| { @@ -107,11 +107,11 @@ mod tests { use crate::collection::AsCollection; let init = input.map(|record| (0, record)); - timely::dataflow::operators::generic::operator::empty(&init.scope()) + timely::dataflow::operators::generic::operator::empty(init.scope()) .as_collection() .iterate(|scope, diff| init.clone() - .enter(&scope) + .enter(scope) .concat(diff) .map(|(round, num)| ((round + num) / 10, (round, num))) .reduce(|_hash, input, output| { diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index e8d1b0f4e..90156d688 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -59,7 +59,7 @@ where // most two elements, then summarizes itself using the `combine` function. Finally, we re-add // the initial `unit_ranges` intervals, so that the set of ranges grows monotonically. - let unit_ranges = unit_ranges.enter(&scope); + let unit_ranges = unit_ranges.enter(scope); ranges .filter(|&((_pos, log, _), _)| log < 64) .map(|((pos, log, key), data)| ((pos >> 1, log + 1, key), (pos, data))) @@ -141,9 +141,9 @@ where init_states .clone() .iterate(|scope, states| { - let init_states = init_states.enter(&scope); + let init_states = init_states.enter(scope); used_ranges - .enter(&scope) + .enter(scope) .map(|((pos, log, key), data)| ((pos << log, key), (log, data))) .join_map(states, move |&(pos, ref key), &(log, ref data), state| ((pos + (1 << log), key.clone()), combine(key, state, data))) diff --git a/differential-dataflow/src/capture.rs b/differential-dataflow/src/capture.rs index 65069bd74..def7c9ae4 100644 --- a/differential-dataflow/src/capture.rs +++ b/differential-dataflow/src/capture.rs @@ -230,7 +230,7 @@ pub mod source { use timely::dataflow::{Scope, Stream, operators::{Capability, CapabilitySet}}; use timely::dataflow::operators::generic::OutputBuilder; use timely::progress::Timestamp; - use timely::scheduling::{Scheduler, SyncActivator}; + use timely::scheduling::SyncActivator; // TODO(guswynn): implement this generally in timely struct DropActivator { @@ -307,11 +307,11 @@ pub mod source { let shared_frontier2 = shared_frontier.clone(); // Step 1: The MESSAGES operator. - let mut messages_op = OperatorBuilder::new("CDCV2_Messages".to_string(), scope.clone()); + let mut messages_op = OperatorBuilder::new("CDCV2_Messages".to_string(), scope); let address = messages_op.operator_info().address; - let activator = scope.sync_activator_for(address.to_vec()); + let activator = scope.worker().sync_activator_for(address.to_vec()); let activator2 = scope.activator_for(Rc::clone(&address)); - let drop_activator = DropActivator { activator: Arc::new(scope.sync_activator_for(address.to_vec())) }; + let drop_activator = DropActivator { activator: Arc::new(scope.worker().sync_activator_for(address.to_vec())) }; let mut source = source_builder(activator); let (updates_out, updates) = messages_op.new_output(); let mut updates_out = OutputBuilder::from(updates_out); @@ -388,7 +388,7 @@ pub mod source { }); // Step 2: The UPDATES operator. - let mut updates_op = OperatorBuilder::new("CDCV2_Updates".to_string(), scope.clone()); + let mut updates_op = OperatorBuilder::new("CDCV2_Updates".to_string(), scope); let mut input = updates_op.new_input(updates, Exchange::new(|x: &(D, T, R)| x.hashed())); let (changes_out, changes) = updates_op.new_output(); let mut changes_out = OutputBuilder::from(changes_out); @@ -437,7 +437,7 @@ pub mod source { }); // Step 3: The PROGRESS operator. - let mut progress_op = OperatorBuilder::new("CDCV2_Progress".to_string(), scope.clone()); + let mut progress_op = OperatorBuilder::new("CDCV2_Progress".to_string(), scope); let mut input = progress_op.new_input( progress, Exchange::new(|x: &(usize, Progress)| x.0 as u64), @@ -521,7 +521,7 @@ pub mod source { }); // Step 4: The FEEDBACK operator. - let mut feedback_op = OperatorBuilder::new("CDCV2_Feedback".to_string(), scope.clone()); + let mut feedback_op = OperatorBuilder::new("CDCV2_Feedback".to_string(), scope); let mut input = feedback_op.new_input( frontier, Exchange::new(|x: &(usize, ChangeBatch)| x.0 as u64), @@ -562,7 +562,6 @@ pub mod sink { use timely::dataflow::Stream; use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::operators::generic::{builder_rc::OperatorBuilder, OutputBuilder}; - use timely::scheduling::Scheduler; use crate::{lattice::Lattice, ExchangeData}; use super::{Writer, Message, Progress}; diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 8d5377498..16755c27e 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -108,7 +108,7 @@ impl<'scope, T: Timestamp, C: Container> Collection<'scope, T, C> { /// /// This method is a specialization of `enter` to the case where the nested scope is a region. /// It removes the need for an operator that adjusts the timestamp. - pub fn enter_region<'inner>(self, child: &Scope<'inner, T>) -> Collection<'inner, T, C> { + pub fn enter_region<'inner>(self, child: Scope<'inner, T>) -> Collection<'inner, T, C> { self.inner .enter(child) .as_collection() @@ -208,13 +208,13 @@ impl<'scope, T: Timestamp, C: Container> Collection<'scope, T, C> { /// let result = scope.region(|child| { /// data.clone() /// .enter(child) - /// .leave(&scope) + /// .leave(scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter<'inner, TInner>(self, child: &Scope<'inner, TInner>) -> Collection<'inner, TInner, >::InnerContainer> + pub fn enter<'inner, TInner>(self, child: Scope<'inner, TInner>) -> Collection<'inner, TInner, >::InnerContainer> where C: containers::Enter, TInner: Refines, @@ -287,7 +287,7 @@ impl<'scope, T: Timestamp, C: Container> Collection<'scope, T, C> /// data.assert_eq(result); /// }); /// ``` - pub fn leave<'outer, TOuter>(self, outer: &Scope<'outer, TOuter>) -> Collection<'outer, TOuter, >::OuterContainer> + pub fn leave<'outer, TOuter>(self, outer: Scope<'outer, TOuter>) -> Collection<'outer, TOuter, >::OuterContainer> where TOuter: Timestamp, T: Refines, @@ -306,7 +306,7 @@ impl<'scope, T: Timestamp, C: Container> Collection<'scope, T, C> /// /// This method is a specialization of `leave` to the case that of a nested region. /// It removes the need for an operator that adjusts the timestamp. - pub fn leave_region<'outer>(self, outer: &Scope<'outer, T>) -> Collection<'outer, T, C> { + pub fn leave_region<'outer>(self, outer: Scope<'outer, T>) -> Collection<'outer, T, C> { self.inner .leave(outer) .as_collection() @@ -533,13 +533,13 @@ pub mod vec { /// let result = scope.iterative::(|child| { /// data.clone() /// .enter_at(child, |x| *x) - /// .leave(&scope) + /// .leave(scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter_at<'inner, TInner, F>(self, child: &Iterative<'inner, T, TInner>, mut initial: F) -> Collection<'inner, Product, D, R> + pub fn enter_at<'inner, TInner, F>(self, child: Iterative<'inner, T, TInner>, mut initial: F) -> Collection<'inner, Product, D, R> where TInner: Timestamp+Hash, F: FnMut(&D) -> TInner + Clone + 'static, @@ -1286,7 +1286,7 @@ impl<'scope, T: Timestamp, C> AsCollection<'scope, T, C> for Stream<'scope, T, C /// .assert_eq(data); /// }); /// ``` -pub fn concatenate<'scope, T, C, I>(scope: &mut Scope<'scope, T>, iterator: I) -> Collection<'scope, T, C> +pub fn concatenate<'scope, T, C, I>(scope: Scope<'scope, T>, iterator: I) -> Collection<'scope, T, C> where T: Timestamp, C: Container, diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index 03045270c..e43323c7d 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -41,7 +41,7 @@ pub trait Input<'scope> : TimelyInput<'scope> { /// /// }).unwrap(); /// ``` - fn new_collection(&mut self) -> (InputSession, VecCollection<'scope, Self::Timestamp, D, R>) + fn new_collection(&self) -> (InputSession, VecCollection<'scope, Self::Timestamp, D, R>) where D: Data, R: Semigroup+'static; /// Create a new collection and input handle from initial data. /// @@ -67,7 +67,7 @@ pub trait Input<'scope> : TimelyInput<'scope> { /// /// }).unwrap(); /// ``` - fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection<'scope, Self::Timestamp, I::Item, isize>) + fn new_collection_from(&self, data: I) -> (InputSession, VecCollection<'scope, Self::Timestamp, I::Item, isize>) where I: IntoIterator + 'static; /// Create a new collection and input handle from initial data. /// @@ -93,24 +93,24 @@ pub trait Input<'scope> : TimelyInput<'scope> { /// /// }).unwrap(); /// ``` - fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection<'scope, Self::Timestamp, D, R>) + fn new_collection_from_raw(&self, data: I) -> (InputSession, VecCollection<'scope, Self::Timestamp, D, R>) where I: IntoIterator+'static, D: Data, R: Semigroup+'static; } use crate::lattice::Lattice; impl<'scope, T: Timestamp + Lattice + timely::order::TotalOrder> Input<'scope> for Scope<'scope, T> { - fn new_collection(&mut self) -> (InputSession, VecCollection<'scope, T, D, R>) + fn new_collection(&self) -> (InputSession, VecCollection<'scope, T, D, R>) where D: Data, R: Semigroup+'static, { let (handle, stream) = self.new_input(); (InputSession::from(handle), stream.as_collection()) } - fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection<'scope, T, I::Item, isize>) + fn new_collection_from(&self, data: I) -> (InputSession, VecCollection<'scope, T, I::Item, isize>) where I: IntoIterator+'static, I::Item: Data { self.new_collection_from_raw(data.into_iter().map(|d| (d, ::minimum(), 1))) } - fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection<'scope, T, D, R>) + fn new_collection_from_raw(&self, data: I) -> (InputSession, VecCollection<'scope, T, D, R>) where D: Data, R: Semigroup+'static, @@ -119,7 +119,7 @@ impl<'scope, T: Timestamp + Lattice + timely::order::TotalOrder> Input<'scope> f use timely::dataflow::operators::ToStream; let (handle, stream) = self.new_input(); - let source = data.to_stream(self).as_collection(); + let source = data.to_stream(*self).as_collection(); (InputSession::from(handle), stream.as_collection().concat(source)) } @@ -199,7 +199,7 @@ impl InputSession { impl InputSession { /// Introduces a handle as collection. - pub fn to_collection<'scope>(&mut self, scope: &mut Scope<'scope, T>) -> VecCollection<'scope, T, D, R> + pub fn to_collection<'scope>(&mut self, scope: Scope<'scope, T>) -> VecCollection<'scope, T, D, R> where T: timely::order::TotalOrder, { diff --git a/differential-dataflow/src/operators/arrange/agent.rs b/differential-dataflow/src/operators/arrange/agent.rs index 61118a497..630c4258a 100644 --- a/differential-dataflow/src/operators/arrange/agent.rs +++ b/differential-dataflow/src/operators/arrange/agent.rs @@ -9,7 +9,6 @@ use timely::dataflow::operators::generic::{OperatorInfo, source}; use timely::progress::Timestamp; use timely::progress::{Antichain, frontier::AntichainRef}; use timely::dataflow::operators::CapabilitySet; -use timely::scheduling::Scheduler; use crate::trace::{Trace, TraceReader, BatchReader}; use crate::trace::wrappers::rc::TraceBox; @@ -216,13 +215,13 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import<'scope>(&mut self, scope: &Scope<'scope, Tr::Time>) -> Arranged<'scope, TraceAgent> + pub fn import<'scope>(&mut self, scope: Scope<'scope, Tr::Time>) -> Arranged<'scope, TraceAgent> { self.import_named(scope, "ArrangedSource") } /// Same as `import`, but allows to name the source. - pub fn import_named<'scope>(&mut self, scope: &Scope<'scope, Tr::Time>, name: &str) -> Arranged<'scope, TraceAgent> + pub fn import_named<'scope>(&mut self, scope: Scope<'scope, Tr::Time>, name: &str) -> Arranged<'scope, TraceAgent> { // Drop ShutdownButton and return only the arrangement. self.import_core(scope, name).0 @@ -275,7 +274,7 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_core<'scope>(&mut self, scope: &Scope<'scope, Tr::Time>, name: &str) -> (Arranged<'scope, TraceAgent>, ShutdownButton>) + pub fn import_core<'scope>(&mut self, scope: Scope<'scope, Tr::Time>, name: &str) -> (Arranged<'scope, TraceAgent>, ShutdownButton>) { let trace = self.clone(); @@ -388,7 +387,7 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_frontier<'scope>(&mut self, scope: &Scope<'scope, Tr::Time>, name: &str) -> (Arranged<'scope, TraceFrontier>>, ShutdownButton>) + pub fn import_frontier<'scope>(&mut self, scope: Scope<'scope, Tr::Time>, name: &str) -> (Arranged<'scope, TraceFrontier>>, ShutdownButton>) where Tr: TraceReader, { @@ -405,7 +404,7 @@ impl TraceAgent { /// /// Invoking this method with an `until` of `Antichain::new()` will perform no filtering, as the empty /// frontier indicates the end of times. - pub fn import_frontier_core<'scope>(&mut self, scope: &Scope<'scope, Tr::Time>, name: &str, since: Antichain, until: Antichain) -> (Arranged<'scope, TraceFrontier>>, ShutdownButton>) + pub fn import_frontier_core<'scope>(&mut self, scope: Scope<'scope, Tr::Time>, name: &str, since: Antichain, until: Antichain) -> (Arranged<'scope, TraceFrontier>>, ShutdownButton>) where Tr: TraceReader, { diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 7435e692e..de533a08f 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -25,8 +25,6 @@ use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; use timely::progress::Antichain; use timely::dataflow::operators::Capability; -use timely::scheduling::Scheduler; -use timely::worker::AsWorker; use crate::{Data, VecCollection, AsCollection}; use crate::difference::Semigroup; @@ -83,7 +81,7 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter<'inner, TInner>(self, child: &Scope<'inner, TInner>) -> Arranged<'inner, TraceEnter> + pub fn enter<'inner, TInner>(self, child: Scope<'inner, TInner>) -> Arranged<'inner, TraceEnter> where TInner: Refines+Lattice, { @@ -97,7 +95,7 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn enter_region<'inner>(self, child: &Scope<'inner, Tr::Time>) -> Arranged<'inner, Tr> { + pub fn enter_region<'inner>(self, child: Scope<'inner, Tr::Time>) -> Arranged<'inner, Tr> { Arranged { stream: self.stream.enter(child), trace: self.trace, @@ -109,7 +107,7 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter_at<'inner, TInner, F, P>(self, child: &Scope<'inner, TInner>, logic: F, prior: P) -> Arranged<'inner, TraceEnterAt> + pub fn enter_at<'inner, TInner, F, P>(self, child: Scope<'inner, TInner>, logic: F, prior: P) -> Arranged<'inner, TraceEnterAt> where TInner: Refines+Lattice+'static, F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone+'static, @@ -314,7 +312,7 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn leave_region<'outer>(self, outer: &Scope<'outer, Tr::Time>) -> Arranged<'outer, Tr> { + pub fn leave_region<'outer>(self, outer: Scope<'outer, Tr::Time>) -> Arranged<'outer, Tr> { use timely::dataflow::operators::Leave; Arranged { stream: self.stream.leave(outer), @@ -383,7 +381,7 @@ where let stream = stream.unary_frontier(pact, name, move |_capability, info| { // Acquire a logger for arrange events. - let logger = scope.logger_for::("differential/arrange").map(Into::into); + let logger = scope.worker().logger_for::("differential/arrange").map(Into::into); // Where we will deposit received updates, and from which we extract batches. let mut batcher = Ba::new(logger.clone(), info.global_id); @@ -394,7 +392,7 @@ where let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); // If there is default exertion logic set, install it. - if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { + if let Some(exert_logic) = scope.worker().config().get::("differential/default_exert_logic").cloned() { empty_trace.set_exert_logic(exert_logic); } diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index 0c88eebd9..5e336fc3d 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -106,8 +106,6 @@ use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::Exchange; use timely::progress::{Antichain, Timestamp}; use timely::dataflow::operators::Capability; -use timely::scheduling::Scheduler; -use timely::worker::AsWorker; use crate::operators::arrange::arrangement::Arranged; use crate::trace::{Builder, Description}; @@ -156,7 +154,7 @@ where stream.unary_frontier(exchange, name, move |_capability, info| { // Acquire a logger for arrange events. - let logger = scope.logger_for::("differential/arrange").map(Into::into); + let logger = scope.worker().logger_for::("differential/arrange").map(Into::into); // Tracks the lower envelope of times in `priority_queue`. let mut capabilities = Antichain::>::new(); @@ -164,7 +162,7 @@ where let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); - if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { + if let Some(exert_logic) = scope.worker().config().get::("differential/default_exert_logic").cloned() { empty_trace.set_exert_logic(exert_logic); } diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index 496f9f5c1..0beae88bf 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -92,9 +92,9 @@ impl<'scope, T: Timestamp + Lattice, D: Ord+Data+Debug, R: Abelian+'static> Iter // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let (variable, collection) = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); - let result = logic(subgraph.clone(), collection); + let result = logic(subgraph, collection); variable.set(result.clone()); - result.leave(&outer) + result.leave(outer) }) } } @@ -104,7 +104,7 @@ impl<'scope, T: Timestamp + Lattice, D: Ord+Data+Debug, R: Semigroup+'static> It where for<'inner> F: FnOnce(Iterative<'inner, T, u64>, VecCollection<'inner, Product, D, R>)->VecCollection<'inner, Product, D, R>, { - let outer = self.clone(); + let outer = self; self.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // @@ -113,9 +113,9 @@ impl<'scope, T: Timestamp + Lattice, D: Ord+Data+Debug, R: Semigroup+'static> It // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let (variable, collection) = Variable::new(subgraph, Product::new(Default::default(), 1)); - let result = logic(subgraph.clone(), collection); + let result = logic(subgraph, collection); variable.set(result.clone()); - result.leave(&outer) + result.leave(outer) } ) } @@ -148,7 +148,7 @@ impl<'scope, T: Timestamp + Lattice, D: Ord+Data+Debug, R: Semigroup+'static> It /// let result = collection.map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate(); /// variable.set(result.clone()); -/// result.leave(&scope) +/// result.leave(scope) /// }); /// }) /// ``` @@ -218,7 +218,7 @@ where /// will produce its fixed point in the outer scope. /// /// In a non-iterative scope the mechanics are the same, but the interpretation varies. - pub fn new(scope: &mut Scope<'scope, T>, step: T::Summary) -> (Self, Collection<'scope, T, C>) { + pub fn new(scope: Scope<'scope, T>, step: T::Summary) -> (Self, Collection<'scope, T, C>) { let (feedback, updates) = scope.feedback(step.clone()); let collection = Collection::::new(updates); (Self { feedback, source: None, step }, collection) diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index b01e3bf89..00277ced3 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -13,7 +13,6 @@ use timely::dataflow::Stream; use timely::dataflow::operators::generic::{Operator, OutputBuilderSession, Session}; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Capability; -use timely::scheduling::Scheduler; use crate::lattice::Lattice; use crate::operators::arrange::Arranged; diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index a695527a8..06fbddc99 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -11,8 +11,6 @@ use timely::progress::frontier::Antichain; use timely::progress::Timestamp; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; -use timely::scheduling::Scheduler; -use timely::worker::AsWorker; use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; @@ -51,12 +49,12 @@ where trace.stream.unary_frontier(Pipeline, name, move |_capability, operator_info| { // Acquire a logger for arrange events. - let logger = scope.logger_for::("differential/arrange").map(Into::into); + let logger = scope.worker().logger_for::("differential/arrange").map(Into::into); let activator = Some(scope.activator_for(operator_info.address.clone())); let mut empty = Tr2::new(operator_info.clone(), logger.clone(), activator); // If there is default exert logic set, install it. - if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { + if let Some(exert_logic) = scope.worker().config().get::("differential/default_exert_logic").cloned() { empty.set_exert_logic(exert_logic); } diff --git a/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index 721f7709d..2362bf6af 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -211,8 +211,8 @@ where // repeatedly update minimal distances each node can be reached from each root nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&scope); - let nodes = nodes.enter(&scope); + let edges = edges.enter(scope); + let nodes = nodes.enter(scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/tests/scc.rs b/differential-dataflow/tests/scc.rs index 2c0b35d92..c1e702ff7 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -219,7 +219,7 @@ where T: timely::progress::Timestamp + Lattice + Ord + Hash, { graph.clone().iterate(|scope, inner| { - let edges = graph.enter(&scope); + let edges = graph.enter(scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); _trim_edges(_trim_edges(inner, edges), trans) }) @@ -250,8 +250,8 @@ where edges.clone() // <-- wth is this. .filter(|_| false) .iterate(|scope, inner| { - let edges = edges.enter(&scope); - let nodes = nodes.enter_at(&scope, |r| 256 * (64 - (r.0 as u64).leading_zeros() as u64)); + let edges = edges.enter(scope); + let nodes = nodes.enter_at(scope, |r| 256 * (64 - (r.0 as u64).leading_zeros() as u64)); inner.join_map(edges, |_k,l,d| (*d,*l)) .concat(nodes) diff --git a/dogsdogsdogs/examples/delta_query.rs b/dogsdogsdogs/examples/delta_query.rs index 74d24b246..bf5388fce 100644 --- a/dogsdogsdogs/examples/delta_query.rs +++ b/dogsdogsdogs/examples/delta_query.rs @@ -89,7 +89,7 @@ fn main() { let changes3 = validate(changes3, reverse_self_alt.clone(), key2.clone()); let changes3 = changes3.map(|((a,c),b)| (a,b,c)); - let prev_changes = changes1.concat(changes2).concat(changes3).leave(&scope); + let prev_changes = changes1.concat(changes2).concat(changes3).leave(scope); // New ideas let d_edges = edges.differentiate(inner); @@ -115,7 +115,7 @@ fn main() { .join_core(forward_key_alt, |a,c,b| Some(((*c, *b), *a))) .join_core(reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c))); - let next_changes = changes1.concat(changes2).concat(changes3).integrate(&scope); + let next_changes = changes1.concat(changes2).concat(changes3).integrate(scope); (prev_changes, next_changes) }); diff --git a/dogsdogsdogs/examples/delta_query_wcoj.rs b/dogsdogsdogs/examples/delta_query_wcoj.rs index 760c0afc1..cba34a71f 100644 --- a/dogsdogsdogs/examples/delta_query_wcoj.rs +++ b/dogsdogsdogs/examples/delta_query_wcoj.rs @@ -79,7 +79,7 @@ fn main() { ]) .map(|((a,c),b)| (a,b,c)); - changes1.concat(changes2).concat(changes3).leave(&scope) + changes1.concat(changes2).concat(changes3).leave(scope) }); triangles diff --git a/dogsdogsdogs/src/calculus.rs b/dogsdogsdogs/src/calculus.rs index a9bfe1589..f7697e1e6 100644 --- a/dogsdogsdogs/src/calculus.rs +++ b/dogsdogsdogs/src/calculus.rs @@ -22,12 +22,12 @@ use crate::altneu::AltNeu; /// Produce a collection containing the changes at the moments they happen. pub trait Differentiate<'scope, T: Timestamp, D: Data, R: Abelian> { - fn differentiate<'inner>(self, child: &Scope<'inner, AltNeu>) -> VecCollection<'inner, AltNeu, D, R>; + fn differentiate<'inner>(self, child: Scope<'inner, AltNeu>) -> VecCollection<'inner, AltNeu, D, R>; } /// Collect instantaneous changes back in to a collection. pub trait Integrate<'scope, T: Timestamp, D: Data, R: Abelian> { - fn integrate<'outer>(self, outer: &Scope<'outer, T>) -> VecCollection<'outer, T, D, R>; + fn integrate<'outer>(self, outer: Scope<'outer, T>) -> VecCollection<'outer, T, D, R>; } impl<'scope, T, D, R> Differentiate<'scope, T, D, R> for VecCollection<'scope, T, D, R> @@ -37,7 +37,7 @@ where R: Abelian + 'static, { // For each (data, Alt(time), diff) we add a (data, Neu(time), -diff). - fn differentiate<'inner>(self, child: &Scope<'inner, AltNeu>) -> VecCollection<'inner, AltNeu, D, R> { + fn differentiate<'inner>(self, child: Scope<'inner, AltNeu>) -> VecCollection<'inner, AltNeu, D, R> { self.enter(child) .inner .flat_map(|(data, time, diff)| { @@ -58,7 +58,7 @@ where R: Abelian + 'static, { // We discard each `neu` variant and strip off the `alt` wrapper. - fn integrate<'outer>(self, outer: &Scope<'outer, T>) -> VecCollection<'outer, T, D, R> { + fn integrate<'outer>(self, outer: Scope<'outer, T>) -> VecCollection<'outer, T, D, R> { self.inner .filter(|(_d,t,_r)| !t.neu) .as_collection() diff --git a/dogsdogsdogs/src/lib.rs b/dogsdogsdogs/src/lib.rs index 91549a6f3..ce484e3b7 100644 --- a/dogsdogsdogs/src/lib.rs +++ b/dogsdogsdogs/src/lib.rs @@ -192,17 +192,17 @@ where type Extension = V; fn count(&mut self, prefixes: VecCollection<'scope, T, (P, usize, usize), R>, index: usize) -> VecCollection<'scope, T, (P, usize, usize), R> { - let counts = self.indices.count_trace.import(&prefixes.scope()); + let counts = self.indices.count_trace.import(prefixes.scope()); operators::count::count(prefixes, counts, self.key_selector.clone(), index) } fn propose(&mut self, prefixes: VecCollection<'scope, T, P, R>) -> VecCollection<'scope, T, (P, V), R> { - let propose = self.indices.propose_trace.import(&prefixes.scope()); + let propose = self.indices.propose_trace.import(prefixes.scope()); operators::propose::propose(prefixes, propose, self.key_selector.clone()) } fn validate(&mut self, extensions: VecCollection<'scope, T, (P, V), R>) -> VecCollection<'scope, T, (P, V), R> { - let validate = self.indices.validate_trace.import(&extensions.scope()); + let validate = self.indices.validate_trace.import(extensions.scope()); operators::validate::validate(extensions, validate, self.key_selector.clone()) } } diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index e095d95a8..36836357c 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -38,7 +38,6 @@ use std::time::Instant; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; use timely::dataflow::Stream; -use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::{Capability, Operator, generic::Session}; use timely::PartialOrder; diff --git a/dogsdogsdogs/src/operators/half_join2.rs b/dogsdogsdogs/src/operators/half_join2.rs index 2cc963f76..2c61b8e3f 100644 --- a/dogsdogsdogs/src/operators/half_join2.rs +++ b/dogsdogsdogs/src/operators/half_join2.rs @@ -23,7 +23,6 @@ use std::ops::Mul; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; use timely::dataflow::Stream; -use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::Operator; use timely::PartialOrder; diff --git a/experiments/src/bin/attend.rs b/experiments/src/bin/attend.rs index 60e17a3f0..a78f8f276 100644 --- a/experiments/src/bin/attend.rs +++ b/experiments/src/bin/attend.rs @@ -24,11 +24,11 @@ fn main() { organizers .clone() .iterate(|scope, attend| { - graph2.enter(&scope) + graph2.enter(scope) .semijoin(attend) .map(|(_,y)| y) .threshold_total(|_,w| if w >= &3 { 1 } else { 0 }) - .concat(organizers.enter(&scope)) + .concat(organizers.enter(scope)) .consolidate() }) .map(|_| ()) diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index c3d2a524a..234152d59 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -217,9 +217,9 @@ fn interactive<'s, G: timely::progress::Timestamp + Lattice>( tc_1.map(|x| (x,x)) .iterate(|scope, inner| edges_q1 - .enter(&scope) + .enter(scope) .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(tc_1_enter.enter(&scope).map(|x| (x,x))) + .concat(tc_1_enter.enter(scope).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -232,9 +232,9 @@ fn interactive<'s, G: timely::progress::Timestamp + Lattice>( .iterate(|scope, inner| edges_q2 .as_collection(|&k,&v| (v,k)) - .enter(&scope) + .enter(scope) .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(tc_2_enter.enter(&scope).map(|x| (x,x))) + .concat(tc_2_enter.enter(scope).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -251,10 +251,10 @@ fn interactive<'s, G: timely::progress::Timestamp + Lattice>( sg_x.iterate(|scope, inner| edges_magic .as_collection(|&k,&v| (v,k)) - .enter(&scope) + .enter(scope) .semijoin(inner) .map(|(_x,y)| y) - .concat(sg_x_enter.enter(&scope)) + .concat(sg_x_enter.enter(scope)) .distinct() ); @@ -271,9 +271,9 @@ fn interactive<'s, G: timely::progress::Timestamp + Lattice>( .map(|x| (x,x)) // for query q, sg(x,x) .iterate(|scope, inner| { - let edges = edges.enter(&scope); - let magic = magic_enter.enter(&scope); - let magic_edges = magic_edges.enter(&scope); + let edges = edges.enter(scope); + let magic = magic_enter.enter(scope); + let magic_edges = magic_edges.enter(scope); let result = inner diff --git a/experiments/src/bin/deals.rs b/experiments/src/bin/deals.rs index 966cc4840..f20e3678f 100644 --- a/experiments/src/bin/deals.rs +++ b/experiments/src/bin/deals.rs @@ -102,7 +102,7 @@ fn tc<'s, T: timely::progress::Timestamp + Lattice + Default + timely::order::Em ; inner.set(result.clone()); - result.leave(&outer) + result.leave(outer) } ) } @@ -132,7 +132,7 @@ fn sg<'s, T: timely::progress::Timestamp + Lattice + Default + timely::order::Em ; inner.set(result.clone()); - result.leave(&outer) + result.leave(outer) } ) } diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 032f5d5ec..838836d5e 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -357,7 +357,7 @@ fn _bidijkstra<'s, T: Timestamp + Lattice + Ord>( reverse.set(reverse_next); - reached.leave(&outer) + reached.leave(outer) }) } @@ -380,8 +380,8 @@ where T: Lattice + std::hash::Hash .filter(|_| false) .iterate(|scope, inner| { - let graph = graph.enter(&scope); - let nodes = nodes.enter_at(&scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); + let graph = graph.enter(scope); + let nodes = nodes.enter_at(scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); let inner = inner.arrange_by_key(); diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index c12e92845..e1f26ad29 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -390,6 +390,6 @@ fn _bidijkstra<'s, T: Timestamp + Lattice + Ord>( reverse.set(reverse_next); - reached.leave(&outer) + reached.leave(outer) }) } diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index d7914c66e..29a6ee6b8 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -294,6 +294,6 @@ fn _bidijkstra<'s, T: Timestamp + Lattice + Ord>( reverse.set(reverse_next); - reached.leave(&outer) + reached.leave(outer) }) } diff --git a/experiments/src/bin/graphs-static.rs b/experiments/src/bin/graphs-static.rs index 02de62956..fc6a5cb3d 100644 --- a/experiments/src/bin/graphs-static.rs +++ b/experiments/src/bin/graphs-static.rs @@ -109,7 +109,7 @@ fn reach<'s>( roots: VecCollection<'s, (), Node, Diff> ) -> VecCollection<'s, (), Node, Diff> { - let graph = graph.import(&roots.scope()); + let graph = graph.import(roots.scope()); let outer = roots.scope(); outer.iterative::(|scope| { @@ -125,7 +125,7 @@ fn reach<'s>( .threshold_total(|_,_| 1); inner.set(result.clone()); - result.leave(&outer) + result.leave(outer) }) } @@ -135,7 +135,7 @@ fn bfs<'s>( roots: VecCollection<'s, (), Node, Diff> ) -> VecCollection<'s, (), (Node, u32), Diff> { - let graph = graph.import(&roots.scope()); + let graph = graph.import(roots.scope()); let roots = roots.map(|r| (r,0)); let outer = roots.scope(); @@ -151,12 +151,12 @@ fn bfs<'s>( .reduce(|_key, input, output| output.push((*input[0].0,1))); inner.set(result.clone()); - result.leave(&outer) + result.leave(outer) }) } fn connected_components<'s>( - scope: &mut timely::dataflow::Scope<'s, ()>, + scope: timely::dataflow::Scope<'s, ()>, forward: &mut TraceHandle, reverse: &mut TraceHandle, ) -> VecCollection<'s, (), (Node, Node), Diff> { diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index c5294a37b..633bb986c 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -91,12 +91,12 @@ fn reach<'s>( roots: VecCollection<'s, (), Node> ) -> VecCollection<'s, (), Node> { - let graph = graph.import(&roots.scope()); + let graph = graph.import(roots.scope()); roots.clone().iterate(|scope, inner| { - let graph = graph.enter(&scope); - let roots = roots.enter(&scope); + let graph = graph.enter(scope); + let roots = roots.enter(scope); // let reach = inner.concat(roots).distinct_total().arrange_by_self(); // graph.join_core(reach, |_src,&dst,&()| Some(dst)) @@ -113,13 +113,13 @@ fn bfs<'s>( roots: VecCollection<'s, (), Node> ) -> VecCollection<'s, (), (Node, u32)> { - let graph = graph.import(&roots.scope()); + let graph = graph.import(roots.scope()); let roots = roots.map(|r| (r,0)); roots.clone().iterate(|scope, inner| { - let graph = graph.enter(&scope); - let roots = roots.enter(&scope); + let graph = graph.enter(scope); + let roots = roots.enter(scope); graph.join_core(inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) .concat(roots) @@ -147,8 +147,8 @@ fn bfs<'s>( // // don't actually use these labels, just grab the type // nodes.filter(|_| false) // .iterate(|scope, inner| { -// let edges = edges.enter(&scope); -// let nodes = nodes.enter_at(&scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); +// let edges = edges.enter(scope); +// let nodes = nodes.enter_at(scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); // inner.join_map(edges, |_k,l,d| (*d,*l)) // .concat(nodes) diff --git a/experiments/src/bin/graspan-interactive.rs b/experiments/src/bin/graspan-interactive.rs index 198a7dfcb..93953bba7 100644 --- a/experiments/src/bin/graspan-interactive.rs +++ b/experiments/src/bin/graspan-interactive.rs @@ -30,8 +30,8 @@ fn main() { .filter(|_| false) .iterate(|scope, inner| { - let nodes = nodes2.enter(&scope); - let edges = edges.enter(&scope); + let nodes = nodes2.enter(scope); + let edges = edges.enter(scope); inner .map(|(a,b)| (b,a)) diff --git a/experiments/src/bin/graspan1.rs b/experiments/src/bin/graspan1.rs index 571b46f1b..ce04f2202 100644 --- a/experiments/src/bin/graspan1.rs +++ b/experiments/src/bin/graspan1.rs @@ -51,7 +51,7 @@ fn main() { .threshold_semigroup(|_,_,x: Option<&Present>| if x.is_none() { Some(Present) } else { None }); labels.set(next.clone()); - next.leave(&outer) + next.leave(outer) }); reached diff --git a/experiments/src/bin/graspan2.rs b/experiments/src/bin/graspan2.rs index f101ae246..fcd05ee65 100644 --- a/experiments/src/bin/graspan2.rs +++ b/experiments/src/bin/graspan2.rs @@ -107,7 +107,7 @@ fn unoptimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(&scope), memory_alias_next.leave(&scope), value_alias_next.leave(&scope)) + (value_flow_next.leave(scope), memory_alias_next.leave(scope), value_alias_next.leave(scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); @@ -232,7 +232,7 @@ fn optimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(&scope), memory_alias_next.leave(&scope)) + (value_flow_next.leave(scope), memory_alias_next.leave(scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); diff --git a/experiments/src/bin/multitemporal.rs b/experiments/src/bin/multitemporal.rs index eb602efe2..f926f5918 100644 --- a/experiments/src/bin/multitemporal.rs +++ b/experiments/src/bin/multitemporal.rs @@ -35,8 +35,8 @@ fn main() { roots.clone().iterate(|scope, inner| { - let edges = edges.enter(&scope); - let roots = roots.enter(&scope); + let edges = edges.enter(scope); + let roots = roots.enter(scope); edges .semijoin(inner) diff --git a/interactive/examples/ddir_col.rs b/interactive/examples/ddir_col.rs index 0d53fb609..6ece2eeae 100644 --- a/interactive/examples/ddir_col.rs +++ b/interactive/examples/ddir_col.rs @@ -131,7 +131,7 @@ mod render { } } - pub fn render_program<'scope>(program: &Program, scope: &mut Scope<'scope, ConcreteTime>, inputs: &[Col<'scope>]) -> HashMap> + pub fn render_program<'scope>(program: &Program, scope: Scope<'scope, ConcreteTime>, inputs: &[Col<'scope>]) -> HashMap> { let mut nodes: HashMap> = HashMap::new(); let mut level: usize = 0; diff --git a/interactive/examples/ddir_vec.rs b/interactive/examples/ddir_vec.rs index 941adc9cb..b3104f223 100644 --- a/interactive/examples/ddir_vec.rs +++ b/interactive/examples/ddir_vec.rs @@ -33,7 +33,7 @@ impl<'scope, T: timely::progress::Timestamp + differential_dataflow::lattice::La } -fn render_program<'scope>(program: &Program, scope: &mut Scope<'scope, DdirTime>, inputs: &[Col<'scope, DdirTime>]) -> HashMap> +fn render_program<'scope>(program: &Program, scope: Scope<'scope, DdirTime>, inputs: &[Col<'scope, DdirTime>]) -> HashMap> { let mut nodes: HashMap> = HashMap::new(); let mut level: usize = 0; diff --git a/mdbook/src/chapter_2/chapter_2_7.md b/mdbook/src/chapter_2/chapter_2_7.md index 18c58965a..80ca6ab8a 100644 --- a/mdbook/src/chapter_2/chapter_2_7.md +++ b/mdbook/src/chapter_2/chapter_2_7.md @@ -53,7 +53,7 @@ In the example above, we could rewrite .clone() .iterate(|scope, transitive| { - let manages = manages.enter(&scope); + let manages = manages.enter(scope); transitive .map(|(mk, m1)| (m1, mk)) @@ -99,7 +99,7 @@ As an example, the implementation of the `iterate` operator looks something like let (variable, collection) = VecVariable::new_from(collection.enter(subgraph), 1); let result = logic(collection); variable.set(result.clone()); - result.leave(&outer) + result.leave(outer) }); # } ``` diff --git a/mdbook/src/chapter_4/chapter_4_1.md b/mdbook/src/chapter_4/chapter_4_1.md index 268367b06..78de41853 100644 --- a/mdbook/src/chapter_4/chapter_4_1.md +++ b/mdbook/src/chapter_4/chapter_4_1.md @@ -15,8 +15,8 @@ Let's write this computation starting from a collection `edges`, using different labels .iterate(|scope, inner| { - let labels = labels.enter(&scope); - let edges = edges.enter(&scope); + let labels = labels.enter(scope); + let edges = edges.enter(scope); inner.join(edges) .map(|(_src,(lbl,dst))| (dst,lbl)) .concat(labels) diff --git a/mdbook/src/chapter_5/chapter_5_4.md b/mdbook/src/chapter_5/chapter_5_4.md index 35ebe9ecd..0f4c6bede 100644 --- a/mdbook/src/chapter_5/chapter_5_4.md +++ b/mdbook/src/chapter_5/chapter_5_4.md @@ -43,8 +43,8 @@ fn main() { // Reachability queries. query.clone().iterate(|scope, reach| { - let knows = knows.enter(&scope); - let query = query.enter(&scope); + let knows = knows.enter(scope); + let query = query.enter(scope); knows.join_core(reach.arrange_by_key(), |x,y,q| [(*y,*q)]) .concat(query) diff --git a/server/dataflows/random_graph/src/lib.rs b/server/dataflows/random_graph/src/lib.rs index cb527b25d..971646c04 100644 --- a/server/dataflows/random_graph/src/lib.rs +++ b/server/dataflows/random_graph/src/lib.rs @@ -3,7 +3,7 @@ use std::cell::RefCell; use rand::{Rng, SeedableRng, StdRng}; -use timely::scheduling::Scheduler; + use timely::dataflow::operators::Probe; use timely::dataflow::operators::generic::operator::source; use timely::progress::Antichain; diff --git a/server/dataflows/reachability/src/lib.rs b/server/dataflows/reachability/src/lib.rs index e5f46ad16..6ad5e9d16 100644 --- a/server/dataflows/reachability/src/lib.rs +++ b/server/dataflows/reachability/src/lib.rs @@ -20,8 +20,8 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() // repeatedly update minimal distances each node can be reached from each root roots.clone().iterate(|scope, dists| { - let edges = edges.enter(&scope); - let roots = roots.enter(&scope); + let edges = edges.enter(scope); + let roots = roots.enter(scope); dists.arrange_by_self() .join_core(edges, |_src, _, &dst| Some(dst)) .concat(roots) diff --git a/server/src/lib.rs b/server/src/lib.rs index b1a58c61f..cdf674b68 100644 --- a/server/src/lib.rs +++ b/server/src/lib.rs @@ -19,7 +19,7 @@ pub type TraceHandle = TraceAgent; /// Arguments provided to each shared library to help build their dataflows and register their results. pub type Environment<'a, 'b> = ( - &'a mut Scope<'b, usize>, + Scope<'b, usize>, &'a mut TraceHandler, &'a mut ProbeHandle, &'a Instant, diff --git a/tpchlike/src/bin/arrange.rs b/tpchlike/src/bin/arrange.rs index a96996bdc..c9ffcd48e 100644 --- a/tpchlike/src/bin/arrange.rs +++ b/tpchlike/src/bin/arrange.rs @@ -59,7 +59,7 @@ fn main() { // use timely::dataflow::ProbeHandle; - let mut context = Context::new(scope.clone(), collections); + let mut context = Context::new(scope, collections); context.index = arrange; diff --git a/tpchlike/src/bin/just-arrange.rs b/tpchlike/src/bin/just-arrange.rs index 60d82de74..ce78720ae 100644 --- a/tpchlike/src/bin/just-arrange.rs +++ b/tpchlike/src/bin/just-arrange.rs @@ -59,7 +59,7 @@ fn main() { // use timely::dataflow::ProbeHandle; - let mut context = Context::new(scope.clone(), collections); + let mut context = Context::new(scope, collections); context.index = arrange; From ddfd05e62129e12250c3fcee73de6df90ef45417 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 13 Apr 2026 17:04:04 -0400 Subject: [PATCH 26/27] Target timely 0.29 (#721) --- Cargo.toml | 4 ++-- advent_of_code_2017/Cargo.toml | 2 +- diagnostics/Cargo.toml | 1 + tpchlike/Cargo.toml | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 96ebd6d9d..9d26900bd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,9 +23,9 @@ rust-version = "1.86" [workspace.dependencies] differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.22.0" } -#timely = { version = "0.28", default-features = false } +timely = { version = "0.29", default-features = false } columnar = { version = "0.12", default-features = false } -timely = { git = "https://github.com/TimelyDataflow/timely-dataflow", default-features = false } +#timely = { git = "https://github.com/TimelyDataflow/timely-dataflow", default-features = false } #timely = { path = "../timely-dataflow/timely/", default-features = false } [workspace.lints.clippy] diff --git a/advent_of_code_2017/Cargo.toml b/advent_of_code_2017/Cargo.toml index 95bdb542b..a120d2cd1 100644 --- a/advent_of_code_2017/Cargo.toml +++ b/advent_of_code_2017/Cargo.toml @@ -11,4 +11,4 @@ workspace = true [dependencies] differential-dataflow = { workspace = true } -timely = { git = "https://github.com/frankmcsherry/timely-dataflow" } +timely = { workspace = true } diff --git a/diagnostics/Cargo.toml b/diagnostics/Cargo.toml index c4a4f89a3..51d7d00d7 100644 --- a/diagnostics/Cargo.toml +++ b/diagnostics/Cargo.toml @@ -1,6 +1,7 @@ [package] name = "diagnostics" version = "0.1.0" +publish = false edition.workspace = true rust-version.workspace = true diff --git a/tpchlike/Cargo.toml b/tpchlike/Cargo.toml index d4b0f20b9..aa23160bf 100644 --- a/tpchlike/Cargo.toml +++ b/tpchlike/Cargo.toml @@ -7,7 +7,7 @@ authors = ["Frank McSherry "] rand = "0.6.5" regex = "0.2" #timely = "0.11" -timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } +timely = { workspace = true } differential-dataflow = { workspace = true } arrayvec = { git = "https://github.com/bluss/arrayvec" } abomonation = "0.7" From 523a501821b4220fddbaf4ce6a715f2fd87287e6 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Fri, 17 Apr 2026 16:43:58 +0200 Subject: [PATCH 27/27] Separate chunker from batcher The chunker was part of the batcher and responsible for transforming input data into the batcher's chain format. Hence, the batcher needed to be aware of its input types, although it would not otherwise use this information. Drop the `Input` and `C` type parameters from `MergeBatcher`, and the `Input` associated type plus `push_container` method from the `Batcher` trait. Batchers now accept chunks via `PushInto`. Chunking moves into `arrange_core`, which gains a `Chu: ContainerBuilder` type parameter so callers can supply a chunker that maps the stream's input container into the batcher's output container. The `Arrange` trait constrains `Ba::Output = C` (same-type chunker) and hardcodes `ContainerChunker` internally, so `.arrange::()` callsites for `Vec`-based collections are unchanged. Callers that need a cross-container chunker (columnar layouts, interactive) drop to `arrange_core` directly. Also updates `chainless_batcher::Batcher` to the new `Batcher` trait shape, and replaces `batcher.push_container(&mut vec\![..])` with `batcher.push_into(vec\![..])` in the trace test. Signed-off-by: Moritz Hoffmann --- .../examples/columnar/columnar_support.rs | 6 +- .../examples/columnar/main.rs | 9 ++- differential-dataflow/src/collection.rs | 21 +++--- .../src/operators/arrange/arrangement.rs | 26 ++++++-- .../implementations/chainless_batcher.rs | 24 +++---- .../trace/implementations/merge_batcher.rs | 64 ++++++------------- .../src/trace/implementations/mod.rs | 1 + .../src/trace/implementations/ord_neu.rs | 5 +- .../src/trace/implementations/rhh.rs | 3 +- differential-dataflow/src/trace/mod.rs | 15 +++-- differential-dataflow/tests/trace.rs | 3 +- interactive/examples/ddir_col.rs | 5 +- 12 files changed, 92 insertions(+), 90 deletions(-) diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index 1241702e7..ae96ac070 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -374,7 +374,7 @@ mod distributor { } } -pub use arrangement::{ValBatcher, ValBuilder, ValSpine}; +pub use arrangement::{ValBatcher, ValBuilder, ValSpine, ValChunker}; pub mod arrangement { use std::rc::Rc; @@ -442,7 +442,9 @@ pub mod arrangement { use crate::{Updates, RecordedUpdates}; use differential_dataflow::trace::implementations::merge_batcher::MergeBatcher; - type ValBatcher2 = MergeBatcher, TrieChunker, trie_merger::TrieMerger>; + type ValBatcher2 = MergeBatcher>; + /// Chunker type for `ValBatcher`: turns `RecordedUpdates` into `Updates` chunks. + pub type ValChunker = TrieChunker; /// A chunker that unwraps `RecordedUpdates` into bare `Updates` for the merge batcher. /// The `records` accounting is discarded here — it has served its purpose for exchange. diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs index 56380089c..9cb3487de 100644 --- a/differential-dataflow/examples/columnar/main.rs +++ b/differential-dataflow/examples/columnar/main.rs @@ -127,13 +127,15 @@ mod reachability { let edges_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; let reach_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let edges_arr = arrange_core::<_, + let edges_arr = arrange_core::<_, _, + ValChunker<(Node, Node, IterTime, Diff)>, ValBatcher, ValBuilder, ValSpine, >(edges_inner.inner, edges_pact, "Edges"); - let reach_arr = arrange_core::<_, + let reach_arr = arrange_core::<_, _, + ValChunker<(Node, (), IterTime, Diff)>, ValBatcher, ValBuilder, ValSpine, @@ -157,7 +159,8 @@ mod reachability { // Arrange for reduce. let combined_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let combined_arr = arrange_core::<_, + let combined_arr = arrange_core::<_, _, + ValChunker<(Node, (), IterTime, Diff)>, ValBatcher, ValBuilder, ValSpine, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 16755c27e..1a6d23024 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -961,9 +961,9 @@ pub mod vec { /// and provide the function `reify` to produce owned keys and values.. pub fn consolidate_named(self, name: &str, reify: F) -> Self where - Ba: crate::trace::Batcher, Time=T> + 'static, + Ba: crate::trace::Batcher, Time=T> + 'static, Tr: for<'a> crate::trace::Trace+'static, - Bu: crate::trace::Builder, + Bu: crate::trace::Builder, Output=Tr::Batch>, F: Fn(Tr::Key<'_>, Tr::Val<'_>) -> D + 'static, { use crate::operators::arrange::arrangement::Arrange; @@ -1018,6 +1018,7 @@ pub mod vec { use crate::trace::implementations::{ValSpine, ValBatcher, ValBuilder}; use crate::trace::implementations::{KeySpine, KeyBatcher, KeyBuilder}; + use crate::trace::implementations::ContainerChunker; use crate::operators::arrange::Arrange; impl<'scope, T, K, V, R> Arrange<'scope, T, Vec<((K, V), T, R)>> for Collection<'scope, T, (K, V), R> @@ -1029,12 +1030,12 @@ pub mod vec { { fn arrange_named(self, name: &str) -> Arranged<'scope, TraceAgent> where - Ba: crate::trace::Batcher, Time=T> + 'static, - Bu: crate::trace::Builder, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, Output = Tr::Batch>, Tr: crate::trace::Trace + 'static, { let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),T,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_, Ba, Bu, _>(self.inner, exchange, name) + crate::operators::arrange::arrangement::arrange_core::<_, _, ContainerChunker>, Ba, Bu, _>(self.inner, exchange, name) } } @@ -1044,12 +1045,12 @@ pub mod vec { { fn arrange_named(self, name: &str) -> Arranged<'scope, TraceAgent> where - Ba: crate::trace::Batcher, Time=T> + 'static, - Bu: crate::trace::Builder, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, Output = Tr::Batch>, Tr: crate::trace::Trace + 'static, { let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),T,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) + crate::operators::arrange::arrangement::arrange_core::<_, _, ContainerChunker>, Ba, Bu, _>(self.map(|k| (k, ())).inner, exchange, name) } } @@ -1069,7 +1070,7 @@ pub mod vec { /// As `arrange_by_key` but with the ability to name the arrangement. pub fn arrange_by_key_named(self, name: &str) -> Arranged<'scope, TraceAgent>> { - self.arrange_named::,ValBuilder<_,_,_,_>,_>(name) + self.arrange_named::, ValBuilder<_,_,_,_>,_>(name) } } @@ -1089,7 +1090,7 @@ pub mod vec { /// As `arrange_by_self` but with the ability to name the arrangement. pub fn arrange_by_self_named(self, name: &str) -> Arranged<'scope, TraceAgent>> { self.map(|k| (k, ())) - .arrange_named::,KeyBuilder<_,_,_>,_>(name) + .arrange_named::, KeyBuilder<_,_,_>,_>(name) } } diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index de533a08f..7d9f88b7d 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -24,6 +24,7 @@ use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; use timely::progress::Antichain; +use timely::container::{ContainerBuilder, PushInto}; use timely::dataflow::operators::Capability; use crate::{Data, VecCollection, AsCollection}; @@ -327,9 +328,13 @@ where T: Timestamp + Lattice, { /// Arranges updates into a shared trace. + /// + /// The batcher's output container must equal the stream container `C`; the default + /// chunker only consolidates same-type containers. For chunker setups that convert + /// between container types (e.g. columnar layouts), call [`arrange_core`] directly. fn arrange(self) -> Arranged<'scope, TraceAgent> where - Ba: Batcher + 'static, + Ba: Batcher + 'static, Bu: Builder, Tr: Trace + 'static, { @@ -337,9 +342,11 @@ where } /// Arranges updates into a shared trace, with a supplied name. + /// + /// See [`Arrange::arrange`] for constraints on the batcher's output container. fn arrange_named(self, name: &str) -> Arranged<'scope, TraceAgent> where - Ba: Batcher + 'static, + Ba: Batcher + 'static, Bu: Builder, Tr: Trace + 'static, ; @@ -350,10 +357,12 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// It uses the supplied parallelization contract to distribute the data, which does not need to /// be consistently by key (though this is the most common). -pub fn arrange_core<'scope, P, Ba, Bu, Tr>(stream: Stream<'scope, Tr::Time, Ba::Input>, pact: P, name: &str) -> Arranged<'scope, TraceAgent> +pub fn arrange_core<'scope, P, C, Chu, Ba, Bu, Tr>(stream: Stream<'scope, Tr::Time, C>, pact: P, name: &str) -> Arranged<'scope, TraceAgent> where - P: ParallelizationContract, - Ba: Batcher + 'static, + C: Container + Clone + 'static, + P: ParallelizationContract, + Chu: ContainerBuilder + for<'a> PushInto<&'a mut C> + 'static, + Ba: Batcher + 'static, Bu: Builder, Tr: Trace+'static, { @@ -403,6 +412,8 @@ where // Initialize to the minimal input frontier. let mut prev_frontier = Antichain::from_elem(Tr::Time::minimum()); + let mut chunker = Chu::default(); + move |(input, frontier), output| { // As we receive data, we need to (i) stash the data and (ii) keep *enough* capabilities. @@ -411,7 +422,10 @@ where input.for_each(|cap, data| { capabilities.insert(cap.retain(0)); - batcher.push_container(data); + chunker.push_into(data); + while let Some(chunk) = chunker.extract() { + batcher.push_into(std::mem::take(chunk)); + } }); // The frontier may have advanced by multiple elements, which is an issue because diff --git a/differential-dataflow/src/trace/implementations/chainless_batcher.rs b/differential-dataflow/src/trace/implementations/chainless_batcher.rs index de4e6c0ee..15d434032 100644 --- a/differential-dataflow/src/trace/implementations/chainless_batcher.rs +++ b/differential-dataflow/src/trace/implementations/chainless_batcher.rs @@ -1,5 +1,6 @@ //! A `Batcher` implementation based on merge sort. +use timely::container::PushInto; use timely::progress::frontier::AntichainRef; use timely::progress::{frontier::Antichain, Timestamp}; @@ -55,9 +56,20 @@ impl> Batcher { } } +impl> PushInto for Batcher { + fn push_into(&mut self, batch: S) { + if batch.len() > 0 { + // TODO: This appears to be optional based on `frontier` only being called after `seal`. + // For the moment, the trait doesn't promise this, but keep eyes on the cost. + batch.lower(&mut self.lower); + self.storages.push(batch); + self.tidy(); + } + } +} + impl> trace::Batcher for Batcher { type Time = T; - type Input = S; type Output = S; fn new(logger: Option, operator_id: usize) -> Self { @@ -70,16 +82,6 @@ impl> trace::Batcher for Batcher { } } - fn push_container(&mut self, batch: &mut Self::Input) { - if batch.len() > 0 { - // TODO: This appears to be optional based on `frontier` only being called after `seal`. - // For the moment, the trait doesn't promise this, but keep eyes on the cost. - batch.lower(&mut self.lower); - self.storages.push(std::mem::take(batch)); - self.tidy(); - } - } - fn seal>(&mut self, upper: Antichain) -> B::Output { let description = trace::Description::new(self.prior.clone(), upper.clone(), Antichain::new()); self.prior = upper.clone(); diff --git a/differential-dataflow/src/trace/implementations/merge_batcher.rs b/differential-dataflow/src/trace/implementations/merge_batcher.rs index d7656fd0e..7716789b6 100644 --- a/differential-dataflow/src/trace/implementations/merge_batcher.rs +++ b/differential-dataflow/src/trace/implementations/merge_batcher.rs @@ -1,31 +1,22 @@ //! A `Batcher` implementation based on merge sort. //! -//! The `MergeBatcher` requires support from two types, a "chunker" and a "merger". -//! The chunker receives input batches and consolidates them, producing sorted output -//! "chunks" that are fully consolidated (no adjacent updates can be accumulated). -//! The merger implements the [`Merger`] trait, and provides hooks for manipulating -//! sorted "chains" of chunks as needed by the merge batcher: merging chunks and also -//! splitting them apart based on time. +//! The `MergeBatcher` requires a "merger" that implements the [`Merger`] trait, which provides +//! hooks for manipulating sorted "chains" of chunks as needed by the merge batcher: merging +//! chunks and also splitting them apart based on time. //! -//! Implementations of `MergeBatcher` can be instantiated through the choice of both -//! the chunker and the merger, provided their respective output and input types align. - -use std::marker::PhantomData; +//! Callers feed already-chunked, sorted-and-consolidated input into the batcher via [`PushInto`]. +//! Forming such chunks from raw data is the responsibility of the caller (typically a chunker +//! living in the surrounding dataflow operator). use timely::progress::frontier::AntichainRef; use timely::progress::{frontier::Antichain, Timestamp}; -use timely::container::{ContainerBuilder, PushInto}; +use timely::container::PushInto; use crate::logging::{BatcherEvent, Logger}; use crate::trace::{Batcher, Builder, Description}; -/// Creates batches from containers of unordered tuples. -/// -/// To implement `Batcher`, the container builder `C` must accept `&mut Input` as inputs, -/// and must produce outputs of type `M::Chunk`. -pub struct MergeBatcher { - /// Transforms input streams to chunks of sorted, consolidated data. - chunker: C, +/// Creates batches from chunks of sorted, consolidated tuples. +pub struct MergeBatcher { /// A sequence of power-of-two length lists of sorted, consolidated containers. /// /// Do not push/pop directly but use the corresponding functions ([`Self::chain_push`]/[`Self::chain_pop`]). @@ -42,16 +33,12 @@ pub struct MergeBatcher { logger: Option, /// Timely operator ID. operator_id: usize, - /// The `Input` type needs to be called out as the type of container accepted, but it is not otherwise present. - _marker: PhantomData, } -impl Batcher for MergeBatcher +impl Batcher for MergeBatcher where - C: ContainerBuilder + for<'a> PushInto<&'a mut Input>, M: Merger, { - type Input = Input; type Time = M::Time; type Output = M::Chunk; @@ -59,23 +46,11 @@ where Self { logger, operator_id, - chunker: C::default(), merger: M::default(), chains: Vec::new(), stash: Vec::new(), frontier: Antichain::new(), lower: Antichain::from_elem(M::Time::minimum()), - _marker: PhantomData, - } - } - - /// Push a container of data into this merge batcher. Updates the internal chain structure if - /// needed. - fn push_container(&mut self, container: &mut Input) { - self.chunker.push_into(container); - while let Some(chunk) = self.chunker.extract() { - let chunk = std::mem::take(chunk); - self.insert_chain(vec![chunk]); } } @@ -84,12 +59,6 @@ where // which we call `lower`, by assumption that after sealing a batcher we receive no more // updates with times not greater or equal to `upper`. fn seal>(&mut self, upper: Antichain) -> B::Output { - // Finish - while let Some(chunk) = self.chunker.finish() { - let chunk = std::mem::take(chunk); - self.insert_chain(vec![chunk]); - } - // Merge all remaining chains into a single chain. while self.chains.len() > 1 { let list1 = self.chain_pop().unwrap(); @@ -125,7 +94,16 @@ where } } -impl MergeBatcher { +impl PushInto for MergeBatcher +where + M: Merger, +{ + fn push_into(&mut self, chunk: M::Chunk) { + self.insert_chain(vec![chunk]); + } +} + +impl MergeBatcher { /// Insert a chain and maintain chain properties: Chains are geometrically sized and ordered /// by decreasing length. fn insert_chain(&mut self, chain: Vec) { @@ -189,7 +167,7 @@ impl MergeBatcher { } } -impl Drop for MergeBatcher { +impl Drop for MergeBatcher { fn drop(&mut self) { // Cleanup chain to retract accounting information. while self.chain_pop().is_some() {} diff --git a/differential-dataflow/src/trace/implementations/mod.rs b/differential-dataflow/src/trace/implementations/mod.rs index 836c6f612..aa8d1c345 100644 --- a/differential-dataflow/src/trace/implementations/mod.rs +++ b/differential-dataflow/src/trace/implementations/mod.rs @@ -48,6 +48,7 @@ pub mod huffman_container; pub mod chunker; // Opinionated takes on default spines. +pub use self::chunker::ContainerChunker; pub use self::ord_neu::OrdValSpine as ValSpine; pub use self::ord_neu::OrdValBatcher as ValBatcher; pub use self::ord_neu::RcOrdValBuilder as ValBuilder; diff --git a/differential-dataflow/src/trace/implementations/ord_neu.rs b/differential-dataflow/src/trace/implementations/ord_neu.rs index 0d5315f6e..980121b14 100644 --- a/differential-dataflow/src/trace/implementations/ord_neu.rs +++ b/differential-dataflow/src/trace/implementations/ord_neu.rs @@ -10,7 +10,6 @@ use std::rc::Rc; -use crate::trace::implementations::chunker::ContainerChunker; use crate::trace::implementations::spine_fueled::Spine; use crate::trace::implementations::merge_batcher::MergeBatcher; use crate::trace::implementations::merge_batcher::container::VecInternalMerger; @@ -24,7 +23,7 @@ pub use self::key_batch::{OrdKeyBatch, OrdKeyBuilder}; /// A trace implementation using a spine of ordered lists. pub type OrdValSpine = Spine>>>; /// A batcher using ordered lists. -pub type OrdValBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, V), T, R>>; +pub type OrdValBatcher = MergeBatcher>; /// A builder using ordered lists. pub type RcOrdValBuilder = RcBuilder, Vec<((K,V),T,R)>>>; @@ -34,7 +33,7 @@ pub type RcOrdValBuilder = RcBuilder = Spine>>>; /// A batcher for ordered lists. -pub type OrdKeyBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, ()), T, R>>; +pub type OrdKeyBatcher = MergeBatcher>; /// A builder for ordered lists. pub type RcOrdKeyBuilder = RcBuilder, Vec<((K,()),T,R)>>>; diff --git a/differential-dataflow/src/trace/implementations/rhh.rs b/differential-dataflow/src/trace/implementations/rhh.rs index 7308b3a49..b12a949df 100644 --- a/differential-dataflow/src/trace/implementations/rhh.rs +++ b/differential-dataflow/src/trace/implementations/rhh.rs @@ -11,7 +11,6 @@ use std::cmp::Ordering; use serde::{Deserialize, Serialize}; use crate::Hashable; -use crate::trace::implementations::chunker::ContainerChunker; use crate::trace::implementations::merge_batcher::MergeBatcher; use crate::trace::implementations::merge_batcher::container::VecInternalMerger; use crate::trace::implementations::spine_fueled::Spine; @@ -24,7 +23,7 @@ use self::val_batch::{RhhValBatch, RhhValBuilder}; /// A trace implementation using a spine of ordered lists. pub type VecSpine = Spine>>>; /// A batcher for ordered lists. -pub type VecBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, V), T, R>>; +pub type VecBatcher = MergeBatcher>; /// A builder for ordered lists. pub type VecBuilder = RcBuilder, Vec<((K,V),T,R)>>>; diff --git a/differential-dataflow/src/trace/mod.rs b/differential-dataflow/src/trace/mod.rs index 582a6da62..7ab47c98d 100644 --- a/differential-dataflow/src/trace/mod.rs +++ b/differential-dataflow/src/trace/mod.rs @@ -12,6 +12,7 @@ pub mod description; pub mod implementations; pub mod wrappers; +use timely::container::PushInto; use timely::progress::{Antichain, frontier::AntichainRef}; use timely::progress::Timestamp; @@ -297,17 +298,17 @@ pub trait Batch : BatchReader + Sized { } /// Functionality for collecting and batching updates. -pub trait Batcher { - /// Type pushed into the batcher. - type Input; - /// Type produced by the batcher. - type Output; +/// +/// Accepts containers of type `Output` via [`PushInto`] and produces output batches of the same +/// type. Callers are responsible for converting raw input data into `Output` containers (e.g. +/// using a chunker) before pushing into the batcher. +pub trait Batcher: PushInto { + /// Type produced by the batcher, and also the type it consumes. + type Output: Default; /// Times at which batches are formed. type Time: Timestamp; /// Allocates a new empty batcher. fn new(logger: Option, operator_id: usize) -> Self; - /// Adds an unordered container of elements to the batcher. - fn push_container(&mut self, batch: &mut Self::Input); /// Returns all updates not greater or equal to an element of `upper`. fn seal>(&mut self, upper: Antichain) -> B::Output; /// Returns the lower envelope of contained update times. diff --git a/differential-dataflow/tests/trace.rs b/differential-dataflow/tests/trace.rs index 54f111a7d..75911e5bc 100644 --- a/differential-dataflow/tests/trace.rs +++ b/differential-dataflow/tests/trace.rs @@ -1,3 +1,4 @@ +use timely::container::PushInto; use timely::dataflow::operators::generic::OperatorInfo; use timely::progress::{Antichain, frontier::AntichainRef}; @@ -14,7 +15,7 @@ fn get_trace() -> ValSpine { { let mut batcher = ValBatcher::::new(None, 0); - batcher.push_container(&mut vec![ + batcher.push_into(vec![ ((1, 2), 0, 1), ((2, 3), 1, 1), ((2, 3), 2, -1), diff --git a/interactive/examples/ddir_col.rs b/interactive/examples/ddir_col.rs index 6ece2eeae..737c1ecd5 100644 --- a/interactive/examples/ddir_col.rs +++ b/interactive/examples/ddir_col.rs @@ -81,6 +81,7 @@ mod columnar { pub type ColValSpine = ValSpine; pub type ColValBatcher = ValBatcher; pub type ColValBuilder = ValBuilder; + pub type ColValChunker = ValChunker; } mod render { @@ -124,8 +125,8 @@ mod render { Rendered::Arrangement(a) => a.clone(), Rendered::Collection(c) => { use differential_dataflow::operators::arrange::arrangement::arrange_core; - use super::columnar::ColValBatcher; - arrange_core::<_, ColValBatcher, ColValBuilder, ColValSpine>(c.inner.clone(), timely::dataflow::channels::pact::Pipeline, "Arrange") + use super::columnar::{ColValBatcher, ColValChunker}; + arrange_core::<_, _, ColValChunker<(Row,Row,Time,Diff)>, ColValBatcher, ColValBuilder, ColValSpine>(c.inner.clone(), timely::dataflow::channels::pact::Pipeline, "Arrange") } } }