diff --git a/container/src/lib.rs b/container/src/lib.rs index 2ff6589fe..7f8447d9d 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -4,63 +4,55 @@ use std::collections::VecDeque; -/// A container transferring data through dataflow edges +/// An type containing a number of records accounted for by progress tracking. /// -/// A container stores a number of elements and thus is able to describe it length (`len()`) and -/// whether it is empty (`is_empty()`). It supports removing all elements (`clear`). -/// -/// A container must implement default. The default implementation is not required to allocate -/// memory for variable-length components. -/// -/// We require the container to be cloneable to enable efficient copies when providing references -/// of containers to operators. Care must be taken that the type's `clone_from` implementation -/// is efficient (which is not necessarily the case when deriving `Clone`.) -pub trait Container: Default { - /// The type of elements when reading non-destructively from the container. - type ItemRef<'a> where Self: 'a; - - /// The type of elements when draining the container. - type Item<'a> where Self: 'a; - - /// Push `item` into self - #[inline] - fn push(&mut self, item: T) where Self: PushInto { - self.push_into(item) - } - - /// The number of elements in this container +/// The object stores a number of updates and thus is able to describe it count +/// (`update_count()`) and whether it is empty (`is_empty()`). It is empty if the +/// update count is zero. +pub trait Accountable { + /// The number of records /// /// This number is used in progress tracking to confirm the receipt of some number /// of outstanding records, and it is highly load bearing. The main restriction is /// imposed on the `LengthPreservingContainerBuilder` trait, whose implementors - /// must preserve the number of items. - fn len(&self) -> usize; + /// must preserve the number of records. + fn record_count(&self) -> i64; - /// Determine if the container contains any elements, corresponding to `len() == 0`. - fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Remove all contents from `self` while retaining allocated memory. - /// After calling `clear`, `is_empty` must return `true` and `len` 0. - fn clear(&mut self); + /// Determine if this contains any updates, corresponding to `update_count() == 0`. + /// It is a correctness error for this to by anything other than `self.record_count() == 0`. + #[inline] fn is_empty(&self) -> bool { self.record_count() == 0 } +} +/// A container that allows iteration morally equivalent to [`IntoIterator`]. +/// +/// Iterating the container presents items in an implementation-specific order. +/// The container's contents are not changed. +pub trait IterContainer { + /// The type of elements when reading non-destructively from the container. + type ItemRef<'a> where Self: 'a; /// Iterator type when reading from the container. type Iter<'a>: Iterator> where Self: 'a; - /// Returns an iterator that reads the contents of this container. fn iter(&self) -> Self::Iter<'_>; +} +/// A container that can drain itself. +/// +/// Draining the container presents items in an implementation-specific order. +/// The container is in an undefined state after calling [`drain`]. Dropping +/// the iterator also leaves the container in an undefined state. +pub trait DrainContainer { + /// The type of elements when draining the container. + type Item<'a> where Self: 'a; /// Iterator type when draining the container. type DrainIter<'a>: Iterator> where Self: 'a; - /// Returns an iterator that drains the contents of this container. /// Drain leaves the container in an undefined state. fn drain(&mut self) -> Self::DrainIter<'_>; } /// A container that can be sized and reveals its capacity. -pub trait SizableContainer: Container { +pub trait SizableContainer { /// Indicates that the container is "full" and should be shipped. fn at_capacity(&self) -> bool; /// Restores `self` to its desired capacity, if it has one. @@ -68,7 +60,10 @@ pub trait SizableContainer: Container { /// The `stash` argument is available, and may have the intended capacity. /// However, it may be non-empty, and may be of the wrong capacity. The /// method should guard against these cases. - fn ensure_capacity(&mut self, stash: &mut Option); + /// + /// Assume that the `stash` is in an undefined state, and properly clear it + /// before re-using it. + fn ensure_capacity(&mut self, stash: &mut Option) where Self: Sized; } /// A container that can absorb items of a specific type. @@ -95,6 +90,9 @@ pub trait PushInto { /// [`Self::finish`]. Implementations can recycle buffers, but should ensure that they clear /// any remaining elements. /// +/// Implementations are allowed to re-use the contents of the mutable references left by the caller, +/// but they should ensure that they clear the contents before doing so. +/// /// For example, a consolidating builder can aggregate differences in-place, but it has /// to ensure that it preserves the intended information. /// @@ -102,7 +100,9 @@ pub trait PushInto { /// decide to represent a push order for `extract` and `finish`, or not. pub trait ContainerBuilder: Default + 'static { /// The container type we're building. - type Container: Container + Clone + 'static; + // The container is `Clone` because `Tee` requires it, otherwise we need to repeat it + // all over Timely. `'static` because we don't want lifetimes everywhere. + type Container: Accountable + Default + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -124,8 +124,8 @@ pub trait ContainerBuilder: Default + 'static { /// A wrapper trait indicating that the container building will preserve the number of records. /// -/// Specifically, the sum of lengths of all extracted and finished containers must equal the -/// number of times that `push_into` is called on the container builder. +/// Specifically, the sum of record counts of all extracted and finished containers must equal the +/// number of accounted records that are pushed into the container builder. /// If you have any questions about this trait you are best off not implementing it. pub trait LengthPreservingContainerBuilder : ContainerBuilder { } @@ -145,14 +145,14 @@ pub struct CapacityContainerBuilder{ pending: VecDeque, } -impl> PushInto for CapacityContainerBuilder { +impl> PushInto for CapacityContainerBuilder { #[inline] fn push_into(&mut self, item: T) { // Ensure capacity self.current.ensure_capacity(&mut self.empty); // Push item - self.current.push(item); + self.current.push_into(item); // Maybe flush if self.current.at_capacity() { @@ -161,7 +161,7 @@ impl> PushInto for CapacityContainerBuil } } -impl ContainerBuilder for CapacityContainerBuilder { +impl ContainerBuilder for CapacityContainerBuilder { type Container = C; #[inline] @@ -184,31 +184,25 @@ impl ContainerBuilder for CapacityContainerBuild } } -impl LengthPreservingContainerBuilder for CapacityContainerBuilder { } +impl LengthPreservingContainerBuilder for CapacityContainerBuilder { } -impl Container for Vec { - type ItemRef<'a> = &'a T where T: 'a; - type Item<'a> = T where T: 'a; - - fn len(&self) -> usize { - Vec::len(self) - } - - fn is_empty(&self) -> bool { - Vec::is_empty(self) - } - - fn clear(&mut self) { Vec::clear(self) } +impl Accountable for Vec { + #[inline] fn record_count(&self) -> i64 { i64::try_from(Vec::len(self)).unwrap() } + #[inline] fn is_empty(&self) -> bool { Vec::is_empty(self) } +} +impl IterContainer for Vec { + type ItemRef<'a> = &'a T where T: 'a; type Iter<'a> = std::slice::Iter<'a, T> where Self: 'a; - - fn iter(&self) -> Self::Iter<'_> { + #[inline] fn iter(&self) -> Self::Iter<'_> { self.as_slice().iter() } +} +impl DrainContainer for Vec { + type Item<'a> = T where T: 'a; type DrainIter<'a> = std::vec::Drain<'a, T> where Self: 'a; - - fn drain(&mut self) -> Self::DrainIter<'_> { + #[inline] fn drain(&mut self) -> Self::DrainIter<'_> { self.drain(..) } } @@ -255,40 +249,21 @@ mod rc { use std::ops::Deref; use std::rc::Rc; - use crate::Container; + use crate::{IterContainer, DrainContainer}; - impl Container for Rc { + impl crate::Accountable for Rc { + #[inline] fn record_count(&self) -> i64 { std::ops::Deref::deref(self).record_count() } + #[inline] fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } + } + impl IterContainer for Rc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; - type Item<'a> = T::ItemRef<'a> where Self: 'a; - - fn len(&self) -> usize { - std::ops::Deref::deref(self).len() - } - - fn is_empty(&self) -> bool { - std::ops::Deref::deref(self).is_empty() - } - - fn clear(&mut self) { - // Try to reuse the allocation if possible - if let Some(inner) = Rc::get_mut(self) { - inner.clear(); - } else { - *self = Self::default(); - } - } - type Iter<'a> = T::Iter<'a> where Self: 'a; - - fn iter(&self) -> Self::Iter<'_> { - self.deref().iter() - } - + #[inline] fn iter(&self) -> Self::Iter<'_> { self.deref().iter() } + } + impl DrainContainer for Rc { + type Item<'a> = T::ItemRef<'a> where Self: 'a; type DrainIter<'a> = T::Iter<'a> where Self: 'a; - - fn drain(&mut self) -> Self::DrainIter<'_> { - self.iter() - } + #[inline] fn drain(&mut self) -> Self::DrainIter<'_> { self.iter() } } } @@ -296,40 +271,21 @@ mod arc { use std::ops::Deref; use std::sync::Arc; - use crate::Container; + use crate::{IterContainer, DrainContainer}; - impl Container for Arc { + impl crate::Accountable for Arc { + #[inline] fn record_count(&self) -> i64 { std::ops::Deref::deref(self).record_count() } + #[inline] fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } + } + impl IterContainer for Arc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; - type Item<'a> = T::ItemRef<'a> where Self: 'a; - - fn len(&self) -> usize { - std::ops::Deref::deref(self).len() - } - - fn is_empty(&self) -> bool { - std::ops::Deref::deref(self).is_empty() - } - - fn clear(&mut self) { - // Try to reuse the allocation if possible - if let Some(inner) = Arc::get_mut(self) { - inner.clear(); - } else { - *self = Self::default(); - } - } - type Iter<'a> = T::Iter<'a> where Self: 'a; - - fn iter(&self) -> Self::Iter<'_> { - self.deref().iter() - } - + #[inline] fn iter(&self) -> Self::Iter<'_> { self.deref().iter() } + } + impl DrainContainer for Arc { + type Item<'a> = T::ItemRef<'a> where Self: 'a; type DrainIter<'a> = T::Iter<'a> where Self: 'a; - - fn drain(&mut self) -> Self::DrainIter<'_> { - self.iter() - } + #[inline] fn drain(&mut self) -> Self::DrainIter<'_> { self.iter() } } } diff --git a/timely/examples/columnar.rs b/timely/examples/columnar.rs index b190cd64e..d858aac79 100644 --- a/timely/examples/columnar.rs +++ b/timely/examples/columnar.rs @@ -1,13 +1,12 @@ //! Wordcount based on the `columnar` crate. -use { - std::collections::HashMap, - timely::{Container, container::CapacityContainerBuilder}, - timely::dataflow::channels::pact::{ExchangeCore, Pipeline}, - timely::dataflow::InputHandleCore, - timely::dataflow::operators::{Inspect, Operator, Probe}, - timely::dataflow::ProbeHandle, -}; +use std::collections::HashMap; + +use timely::container::{IterContainer, CapacityContainerBuilder}; +use timely::dataflow::channels::pact::{ExchangeCore, Pipeline}; +use timely::dataflow::InputHandleCore; +use timely::dataflow::operators::{Inspect, Operator, Probe}; +use timely::dataflow::ProbeHandle; // Creates `WordCountContainer` and `WordCountReference` structs, // as well as various implementations relating them to `WordCount`. @@ -177,21 +176,16 @@ mod container { } } - impl timely::Container for Column { - fn len(&self) -> usize { self.borrow().len() } - // This sets `self` to be an empty `Typed` variant, appropriate for pushing into. - fn clear(&mut self) { - match self { - Column::Typed(t) => t.clear(), - Column::Bytes(_) => *self = Column::Typed(Default::default()), - Column::Align(_) => *self = Column::Typed(Default::default()), - } - } - + impl timely::Accountable for Column { + #[inline] fn record_count(&self) -> i64 { i64::try_from(self.borrow().len()).unwrap() } + #[inline] fn is_empty(&self) -> bool { self.borrow().is_empty() } + } + impl timely::container::IterContainer for Column { type ItemRef<'a> = C::Ref<'a>; type Iter<'a> = IterOwn>; fn iter<'a>(&'a self) -> Self::Iter<'a> { self.borrow().into_index_iter() } - + } + impl timely::container::DrainContainer for Column { type Item<'a> = C::Ref<'a>; type DrainIter<'a> = IterOwn>; fn drain<'a>(&'a mut self) -> Self::DrainIter<'a> { self.borrow().into_index_iter() } diff --git a/timely/src/dataflow/channels/mod.rs b/timely/src/dataflow/channels/mod.rs index 6a351753b..36fb44bb1 100644 --- a/timely/src/dataflow/channels/mod.rs +++ b/timely/src/dataflow/channels/mod.rs @@ -39,7 +39,7 @@ impl Message { } /// Forms a message, and pushes contents at `pusher`. Replaces `buffer` with what the pusher - /// leaves in place, or the container's default element. The buffer is cleared. + /// leaves in place, or the container's default element. The buffer is left in an undefined state. #[inline] pub fn push_at>>(buffer: &mut C, time: T, pusher: &mut P) { @@ -51,7 +51,6 @@ impl Message { if let Some(message) = bundle { *buffer = message.data; - buffer.clear(); } } } diff --git a/timely/src/dataflow/channels/pact.rs b/timely/src/dataflow/channels/pact.rs index bb99cd277..96aaed605 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -10,7 +10,8 @@ use std::{fmt::{self, Debug}, marker::PhantomData}; use std::rc::Rc; -use crate::{Container, container::{ContainerBuilder, LengthPreservingContainerBuilder, SizableContainer, CapacityContainerBuilder, PushInto}}; +use crate::Accountable; +use crate::container::{ContainerBuilder, DrainContainer, LengthPreservingContainerBuilder, SizableContainer, CapacityContainerBuilder, PushInto}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; use crate::communication::{Push, Pull}; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; @@ -19,7 +20,6 @@ use crate::dataflow::channels::Message; use crate::logging::{TimelyLogger as Logger, MessagesEvent}; use crate::progress::Timestamp; use crate::worker::AsWorker; -use crate::Data; /// A `ParallelizationContract` allocates paired `Push` and `Pull` implementors. pub trait ParallelizationContract { @@ -35,7 +35,7 @@ pub trait ParallelizationContract { #[derive(Debug)] pub struct Pipeline; -impl ParallelizationContract for Pipeline { +impl ParallelizationContract for Pipeline { type Pusher = LogPusher>>; type Puller = LogPuller>>; fn connect(self, allocator: &mut A, identifier: usize, address: Rc<[usize]>, logging: Option) -> (Self::Pusher, Self::Puller) { @@ -54,7 +54,8 @@ pub type Exchange = ExchangeCore>, F>; impl ExchangeCore where CB: LengthPreservingContainerBuilder, - for<'a> F: FnMut(&::Item<'a>)->u64 + CB::Container: DrainContainer, + for<'a> F: FnMut(&::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. pub fn new_core(func: F) -> ExchangeCore { @@ -67,7 +68,7 @@ where impl ExchangeCore, F> where - C: SizableContainer, + C: SizableContainer + DrainContainer, for<'a> F: FnMut(&C::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. @@ -82,9 +83,9 @@ where // Exchange uses a `Box` because it cannot know what type of pushable will return from the allocator. impl ParallelizationContract for ExchangeCore where - CB: ContainerBuilder + for<'a> PushInto<::Item<'a>>, - CB::Container: Data + Send + crate::dataflow::channels::ContainerBytes, - for<'a> H: FnMut(&::Item<'a>) -> u64 + 'static, + CB: ContainerBuilder + for<'a> PushInto<::Item<'a>>, + CB::Container: Send + crate::dataflow::channels::ContainerBytes, + for<'a> H: FnMut(&::Item<'a>) -> u64 + 'static, { type Pusher = ExchangePusher< T, @@ -132,7 +133,7 @@ impl

LogPusher

{ } } -impl>> Push> for LogPusher

{ +impl>> Push> for LogPusher

{ #[inline] fn push(&mut self, pair: &mut Option>) { if let Some(bundle) = pair { @@ -150,7 +151,7 @@ impl>> Push> for LogPusher< source: self.source, target: self.target, seq_no: self.counter - 1, - length: bundle.data.len(), + record_count: bundle.data.record_count(), }) } } @@ -180,7 +181,7 @@ impl

LogPuller

{ } } -impl>> Pull> for LogPuller

{ +impl>> Pull> for LogPuller

{ #[inline] fn pull(&mut self) -> &mut Option> { let result = self.puller.pull(); @@ -195,7 +196,7 @@ impl>> Pull> for LogPuller< source: bundle.from, target, seq_no: bundle.seq, - length: bundle.data.len(), + record_count: bundle.data.record_count(), }); } } diff --git a/timely/src/dataflow/channels/pullers/counter.rs b/timely/src/dataflow/channels/pullers/counter.rs index 6f71848a4..5903b2e4c 100644 --- a/timely/src/dataflow/channels/pullers/counter.rs +++ b/timely/src/dataflow/channels/pullers/counter.rs @@ -6,10 +6,10 @@ use std::cell::RefCell; use crate::dataflow::channels::Message; use crate::progress::ChangeBatch; use crate::communication::Pull; -use crate::Container; +use crate::Accountable; /// A wrapper which accounts records pulled past in a shared count map. -pub struct Counter>> { +pub struct Counter { pullable: P, consumed: Rc>>, phantom: ::std::marker::PhantomData, @@ -19,7 +19,7 @@ pub struct Counter>> { pub struct ConsumedGuard { consumed: Rc>>, time: Option, - len: usize, + record_count: i64, } impl ConsumedGuard { @@ -32,11 +32,11 @@ impl Drop for ConsumedGuard { fn drop(&mut self) { // SAFETY: we're in a Drop impl, so this runs at most once let time = self.time.take().unwrap(); - self.consumed.borrow_mut().update(time, self.len as i64); + self.consumed.borrow_mut().update(time, self.record_count); } } -impl>> Counter { +impl>> Counter { /// Retrieves the next timestamp and batch of data. #[inline] pub fn next(&mut self) -> Option<&mut Message> { @@ -49,7 +49,7 @@ impl>> Counter let guard = ConsumedGuard { consumed: Rc::clone(&self.consumed), time: Some(message.time.clone()), - len: message.data.len(), + record_count: message.data.record_count(), }; Some((guard, message)) } diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index be579936c..c99d76060 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -6,7 +6,7 @@ use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; -use crate::{Container, Data}; +use crate::{Container, Accountable}; /// Buffers data sent at the same time, for efficient communication. /// @@ -44,7 +44,7 @@ impl Buffer { } } -impl>> Buffer, P> where T: Eq+Clone { +impl>> Buffer, P> where T: Eq+Clone { /// Returns a `Session`, which accepts data to send at the associated time #[inline] pub fn session(&mut self, time: &T) -> Session<'_, T, CapacityContainerBuilder, P> { diff --git a/timely/src/dataflow/channels/pushers/counter.rs b/timely/src/dataflow/channels/pushers/counter.rs index a6a16ce5d..67d39318d 100644 --- a/timely/src/dataflow/channels/pushers/counter.rs +++ b/timely/src/dataflow/channels/pushers/counter.rs @@ -7,7 +7,7 @@ use std::cell::RefCell; use crate::progress::{ChangeBatch, Timestamp}; use crate::dataflow::channels::Message; use crate::communication::Push; -use crate::Container; +use crate::Accountable; /// A wrapper which updates shared `produced` based on the number of records pushed. #[derive(Debug)] @@ -17,11 +17,11 @@ pub struct Counter>> { phantom: PhantomData, } -impl Push> for Counter where P: Push> { +impl Push> for Counter where P: Push> { #[inline] fn push(&mut self, message: &mut Option>) { if let Some(message) = message { - self.produced.borrow_mut().update(message.time.clone(), message.data.len() as i64); + self.produced.borrow_mut().update(message.time.clone(), message.data.record_count()); } // only propagate `None` if dirty (indicates flush) diff --git a/timely/src/dataflow/channels/pushers/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index e72085b4f..a9ea9b4fa 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -1,9 +1,8 @@ //! The exchange pattern distributes pushed data between many target pushees. use crate::communication::Push; -use crate::container::{ContainerBuilder, PushInto}; +use crate::container::{ContainerBuilder, DrainContainer, PushInto}; use crate::dataflow::channels::Message; -use crate::Container; /// Distribute containers to several pushers. /// @@ -43,8 +42,8 @@ impl DrainContainerDistributor { impl Distributor for DrainContainerDistributor where - CB: ContainerBuilder + for<'a> PushInto<::Item<'a>>, - for<'a> H: FnMut(&::Item<'a>) -> u64, + CB: ContainerBuilder + for<'a> PushInto<::Item<'a>>, + for<'a> H: FnMut(&::Item<'a>) -> u64, { fn partition>>(&mut self, container: &mut CB::Container, time: &T, pushers: &mut [P]) { debug_assert_eq!(self.builders.len(), pushers.len()); diff --git a/timely/src/dataflow/channels/pushers/tee.rs b/timely/src/dataflow/channels/pushers/tee.rs index 49d9e17d0..df2e2090d 100644 --- a/timely/src/dataflow/channels/pushers/tee.rs +++ b/timely/src/dataflow/channels/pushers/tee.rs @@ -17,7 +17,7 @@ pub struct Tee { shared: PushList, } -impl Push> for Tee { +impl Push> for Tee { #[inline] fn push(&mut self, message: &mut Option>) { let mut pushers = self.shared.borrow_mut(); @@ -32,14 +32,14 @@ impl Push> for Tee { pushers[index-1].push(&mut None); } } - if pushers.len() > 0 { + if !pushers.is_empty() { let last = pushers.len() - 1; pushers[last].push(message); } } } -impl Tee { +impl Tee { /// Allocates a new pair of `Tee` and `TeeHelper`. pub fn new() -> (Tee, TeeHelper) { let shared = Rc::new(RefCell::new(Vec::new())); @@ -52,7 +52,7 @@ impl Tee { } } -impl Clone for Tee { +impl Clone for Tee { fn clone(&self) -> Self { Self { buffer: Default::default(), diff --git a/timely/src/dataflow/operators/branch.rs b/timely/src/dataflow/operators/branch.rs index 924bd196a..dad88be2b 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -93,7 +93,7 @@ pub trait BranchWhen: Sized { fn branch_when(&self, condition: impl Fn(&T) -> bool + 'static) -> (Self, Self); } -impl BranchWhen for StreamCore { +impl BranchWhen for StreamCore { fn branch_when(&self, condition: impl Fn(&S::Timestamp) -> bool + 'static) -> (Self, Self) { let mut builder = OperatorBuilder::new("Branch".to_owned(), self.scope()); builder.set_notify(false); diff --git a/timely/src/dataflow/operators/core/capture/capture.rs b/timely/src/dataflow/operators/core/capture/capture.rs index 410fe486e..4d8e3053e 100644 --- a/timely/src/dataflow/operators/core/capture/capture.rs +++ b/timely/src/dataflow/operators/core/capture/capture.rs @@ -10,14 +10,14 @@ use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; -use crate::{Container, Data}; +use crate::Container; use crate::progress::ChangeBatch; use crate::progress::Timestamp; use super::{Event, EventPusher}; /// Capture a stream of timestamped data for later replay. -pub trait Capture { +pub trait Capture { /// Captures a stream of timestamped data for later replay. /// /// # Examples @@ -113,7 +113,7 @@ pub trait Capture { } } -impl Capture for StreamCore { +impl Capture for StreamCore { fn capture_into+'static>(&self, mut event_pusher: P) { let mut builder = OperatorBuilder::new("Capture".to_owned(), self.scope()); diff --git a/timely/src/dataflow/operators/core/capture/extract.rs b/timely/src/dataflow/operators/core/capture/extract.rs index fcf024f31..c0af57d37 100644 --- a/timely/src/dataflow/operators/core/capture/extract.rs +++ b/timely/src/dataflow/operators/core/capture/extract.rs @@ -1,7 +1,8 @@ //! Traits and types for extracting captured timely dataflow streams. use super::Event; -use crate::{container::{SizableContainer, PushInto}}; +use crate::Container; +use crate::container::{SizableContainer, DrainContainer, PushInto}; /// Supports extracting a sequence of timestamp and data. pub trait Extract { @@ -50,8 +51,7 @@ pub trait Extract { impl Extract for ::std::sync::mpsc::Receiver> where - for<'a> C: PushInto>, - for<'a> C::Item<'a>: Ord, + for<'a> C: Container + DrainContainer: Ord> + PushInto>, { fn extract(self) -> Vec<(T, C)> { let mut staged = std::collections::BTreeMap::new(); @@ -71,7 +71,7 @@ where to_sort.sort(); let mut sorted = C::default(); for datum in to_sort.into_iter() { - sorted.push(datum); + sorted.push_into(datum); } if !sorted.is_empty() { result.push((time, sorted)); diff --git a/timely/src/dataflow/operators/core/capture/replay.rs b/timely/src/dataflow/operators/core/capture/replay.rs index 49878b887..41ff50d12 100644 --- a/timely/src/dataflow/operators/core/capture/replay.rs +++ b/timely/src/dataflow/operators/core/capture/replay.rs @@ -62,7 +62,7 @@ pub trait Replay : Sized { fn replay_core>(self, scope: &mut S, period: Option) -> StreamCore; } -impl Replay for I +impl Replay for I where I : IntoIterator, ::Item: EventIterator+'static, diff --git a/timely/src/dataflow/operators/core/concat.rs b/timely/src/dataflow/operators/core/concat.rs index 24fcef532..f377d3626 100644 --- a/timely/src/dataflow/operators/core/concat.rs +++ b/timely/src/dataflow/operators/core/concat.rs @@ -1,12 +1,11 @@ //! Merges the contents of multiple streams. - -use crate::{Container, Data}; +use crate::Container; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{StreamCore, Scope}; /// Merge the contents of two streams. -pub trait Concat { +pub trait Concat { /// Merge the contents of two streams. /// /// # Examples @@ -23,14 +22,14 @@ pub trait Concat { fn concat(&self, _: &StreamCore) -> StreamCore; } -impl Concat for StreamCore { +impl Concat for StreamCore { fn concat(&self, other: &StreamCore) -> StreamCore { self.scope().concatenate([self.clone(), other.clone()]) } } /// Merge the contents of multiple streams. -pub trait Concatenate { +pub trait Concatenate { /// Merge the contents of multiple streams. /// /// # Examples @@ -52,7 +51,7 @@ pub trait Concatenate { I: IntoIterator>; } -impl Concatenate for StreamCore { +impl Concatenate for StreamCore { fn concatenate(&self, sources: I) -> StreamCore where I: IntoIterator> @@ -62,7 +61,7 @@ impl Concatenate for StreamCore { } } -impl Concatenate for G { +impl Concatenate for G { fn concatenate(&self, sources: I) -> StreamCore where I: IntoIterator> diff --git a/timely/src/dataflow/operators/core/enterleave.rs b/timely/src/dataflow/operators/core/enterleave.rs index bc221c4a6..c0889498e 100644 --- a/timely/src/dataflow/operators/core/enterleave.rs +++ b/timely/src/dataflow/operators/core/enterleave.rs @@ -26,7 +26,7 @@ use crate::logging::{TimelyLogger, MessagesEvent}; use crate::progress::Timestamp; use crate::progress::timestamp::Refines; use crate::progress::{Source, Target}; -use crate::{Container, Data}; +use crate::{Accountable, Container}; use crate::communication::Push; use crate::dataflow::channels::pushers::{Counter, Tee}; use crate::dataflow::channels::Message; @@ -35,7 +35,7 @@ use crate::dataflow::{StreamCore, Scope}; use crate::dataflow::scopes::Child; /// Extension trait to move a `Stream` into a child of its current `Scope`. -pub trait Enter, C: Container> { +pub trait Enter, C> { /// Moves the `Stream` argument into a child of its current `Scope`. /// /// # Examples @@ -53,7 +53,7 @@ pub trait Enter, C: Container> { fn enter<'a>(&self, _: &Child<'a, G, T>) -> StreamCore, C>; } -impl, C: Data+Container> Enter for StreamCore { +impl, C: Container> Enter for StreamCore { fn enter<'a>(&self, scope: &Child<'a, G, T>) -> StreamCore, C> { use crate::scheduling::Scheduler; @@ -85,7 +85,7 @@ impl, C: Data+Container> Enter { +pub trait Leave { /// Moves a `Stream` to the parent of its current `Scope`. /// /// # Examples @@ -103,7 +103,7 @@ pub trait Leave { fn leave(&self) -> StreamCore; } -impl> Leave for StreamCore, C> { +impl> Leave for StreamCore, C> { fn leave(&self) -> StreamCore { let scope = self.scope(); @@ -130,14 +130,14 @@ impl> Leave, TContainer: Container + Data> { +struct IngressNub, TContainer: Container> { targets: Counter>, phantom: ::std::marker::PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TContainer: Container + Data> Push> for IngressNub { +impl, TContainer: Container> Push> for IngressNub { fn push(&mut self, element: &mut Option>) { if let Some(outer_message) = element { let data = ::std::mem::take(&mut outer_message.data); @@ -159,13 +159,13 @@ impl, TContainer: Container } -struct EgressNub, TContainer: Data> { +struct EgressNub, TContainer> { targets: Tee, phantom: PhantomData, } impl Push> for EgressNub -where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: Data { +where TOuter: Timestamp, TInner: Timestamp+Refines, { fn push(&mut self, message: &mut Option>) { if let Some(inner_message) = message { let data = ::std::mem::take(&mut inner_message.data); @@ -207,7 +207,7 @@ impl

LogPusher

{ impl Push> for LogPusher

where - C: Container, + C: Accountable, P: Push>, { fn push(&mut self, element: &mut Option>) { @@ -218,7 +218,7 @@ where source: self.index, target: self.index, seq_no: self.counter, - length: bundle.data.len(), + record_count: bundle.data.record_count(), }; let recv_event = MessagesEvent { is_send: false, diff --git a/timely/src/dataflow/operators/core/exchange.rs b/timely/src/dataflow/operators/core/exchange.rs index bb021a5d5..92c9d78f5 100644 --- a/timely/src/dataflow/operators/core/exchange.rs +++ b/timely/src/dataflow/operators/core/exchange.rs @@ -1,13 +1,13 @@ //! Exchange records between workers. -use crate::ExchangeData; -use crate::container::{Container, SizableContainer, PushInto}; +use crate::Container; +use crate::container::{DrainContainer, SizableContainer, PushInto}; use crate::dataflow::channels::pact::ExchangeCore; use crate::dataflow::operators::generic::operator::Operator; use crate::dataflow::{Scope, StreamCore}; /// Exchange records between workers. -pub trait Exchange { +pub trait Exchange { /// Exchange records between workers. /// /// The closure supplied should map a reference to a record to a `u64`, @@ -30,9 +30,12 @@ pub trait Exchange { impl Exchange for StreamCore where - C: SizableContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, - C: for<'a> PushInto>, - + C: Container + + SizableContainer + + DrainContainer + + Send + + crate::dataflow::channels::ContainerBytes + + for<'a> PushInto>, { fn exchange(&self, route: F) -> StreamCore where diff --git a/timely/src/dataflow/operators/core/feedback.rs b/timely/src/dataflow/operators/core/feedback.rs index 61bd5c196..067ea3db8 100644 --- a/timely/src/dataflow/operators/core/feedback.rs +++ b/timely/src/dataflow/operators/core/feedback.rs @@ -1,6 +1,6 @@ //! Create cycles in a timely dataflow graph. -use crate::{Container, Data}; +use crate::Container; use crate::container::CapacityContainerBuilder; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pushers::Tee; @@ -36,7 +36,7 @@ pub trait Feedback { /// .connect_loop(handle); /// }); /// ``` - fn feedback(&mut self, summary: ::Summary) -> (Handle, StreamCore); + fn feedback(&mut self, summary: ::Summary) -> (Handle, StreamCore); } /// Creates a `StreamCore` and a `Handle` to later bind the source of that `StreamCore`. @@ -64,12 +64,12 @@ pub trait LoopVariable<'a, G: Scope, T: Timestamp> { /// }); /// }); /// ``` - fn loop_variable(&mut self, summary: T::Summary) -> (Handle, C>, StreamCore, C>); + fn loop_variable(&mut self, summary: T::Summary) -> (Handle, C>, StreamCore, C>); } impl Feedback for G { - fn feedback(&mut self, summary: ::Summary) -> (Handle, StreamCore) { + fn feedback(&mut self, summary: ::Summary) -> (Handle, StreamCore) { let mut builder = OperatorBuilder::new("Feedback".to_owned(), self.clone()); builder.set_notify(false); @@ -80,13 +80,13 @@ impl Feedback for G { } impl<'a, G: Scope, T: Timestamp> LoopVariable<'a, G, T> for Iterative<'a, G, T> { - fn loop_variable(&mut self, summary: T::Summary) -> (Handle, C>, StreamCore, C>) { + fn loop_variable(&mut self, summary: T::Summary) -> (Handle, C>, StreamCore, C>) { self.feedback(Product::new(Default::default(), summary)) } } /// Connect a `Stream` to the input of a loop variable. -pub trait ConnectLoop { +pub trait ConnectLoop { /// Connect a `Stream` to be the input of a loop variable. /// /// # Examples @@ -107,7 +107,7 @@ pub trait ConnectLoop { fn connect_loop(&self, handle: Handle); } -impl ConnectLoop for StreamCore { +impl ConnectLoop for StreamCore { fn connect_loop(&self, handle: Handle) { let mut builder = handle.builder; @@ -132,7 +132,7 @@ impl ConnectLoop for StreamCore { /// A handle used to bind the source of a loop variable. #[derive(Debug)] -pub struct Handle { +pub struct Handle { builder: OperatorBuilder, summary: ::Summary, output: OutputWrapper, Tee>, diff --git a/timely/src/dataflow/operators/core/filter.rs b/timely/src/dataflow/operators/core/filter.rs index 80ef564b0..f82a76339 100644 --- a/timely/src/dataflow/operators/core/filter.rs +++ b/timely/src/dataflow/operators/core/filter.rs @@ -1,12 +1,12 @@ //! Filters a stream by a predicate. -use crate::container::{Container, SizableContainer, PushInto}; -use crate::Data; +use crate::container::{DrainContainer, SizableContainer, PushInto}; +use crate::Container; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::operators::generic::operator::Operator; /// Extension trait for filtering. -pub trait Filter { +pub trait Filter { /// Returns a new instance of `self` containing only records satisfying `predicate`. /// /// # Examples @@ -23,16 +23,14 @@ pub trait Filter { fn filter)->bool+'static>(&self, predicate: P) -> Self; } -impl Filter for StreamCore +impl Filter for StreamCore where for<'a> C: PushInto> { fn filter)->bool+'static>(&self, mut predicate: P) -> StreamCore { self.unary(Pipeline, "Filter", move |_,_| move |input, output| { input.for_each(|time, data| { - if !data.is_empty() { - output.session(&time).give_iterator(data.drain().filter(&mut predicate)); - } + output.session(&time).give_iterator(data.drain().filter(&mut predicate)); }); }) } diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index 5088f6eba..8db72755e 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -10,7 +10,7 @@ use crate::scheduling::{Schedule, Activator}; use crate::progress::{Operate, operate::SharedProgress, Timestamp, ChangeBatch}; use crate::progress::Source; use crate::progress::operate::Connectivity; -use crate::{Container, Data}; +use crate::{Accountable, Container}; use crate::communication::Push; use crate::dataflow::{Scope, ScopeParent, StreamCore}; use crate::dataflow::channels::pushers::{Tee, Counter}; @@ -59,7 +59,7 @@ pub trait Input : Scope { /// } /// }); /// ``` - fn new_input(&mut self) -> (Handle<::Timestamp, CapacityContainerBuilder>, StreamCore); + fn new_input(&mut self) -> (Handle<::Timestamp, CapacityContainerBuilder>, StreamCore); /// Create a new [StreamCore] and [Handle] through which to supply input. /// @@ -134,7 +134,7 @@ pub trait Input : Scope { use crate::order::TotalOrder; impl Input for G where ::Timestamp: TotalOrder { - fn new_input(&mut self) -> (Handle<::Timestamp, CapacityContainerBuilder>, StreamCore) { + fn new_input(&mut self) -> (Handle<::Timestamp, CapacityContainerBuilder>, StreamCore) { let mut handle = Handle::new(); let stream = self.input_from(&mut handle); (handle, stream) @@ -224,7 +224,7 @@ pub struct Handle { now_at: T, } -impl Handle> { +impl Handle> { /// Allocates a new input handle, from which one can create timely streams. /// /// # Examples @@ -374,7 +374,7 @@ impl Handle { /// Sends a container at each of the destinations. There can be more than one; clone if needed. /// Does not take `self` because `flush` and `extract` borrow `self` mutably. - /// Clears the container. + /// Leaves the container in an undefined state. // TODO: Find a better name for this function. #[inline] fn send_container( @@ -392,7 +392,6 @@ impl Handle { Message::push_at(container, now_at.clone(), &mut pushers[index]); } } - container.clear(); } /// Closes the current epoch, flushing if needed, shutting if needed, and updating the frontier. diff --git a/timely/src/dataflow/operators/core/inspect.rs b/timely/src/dataflow/operators/core/inspect.rs index b8c41f97b..6491b9d54 100644 --- a/timely/src/dataflow/operators/core/inspect.rs +++ b/timely/src/dataflow/operators/core/inspect.rs @@ -1,12 +1,13 @@ //! Extension trait and implementation for observing and action on streamed data. -use crate::{Container, Data}; +use crate::Container; +use crate::container::IterContainer; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::operators::generic::Operator; /// Methods to inspect records and batches of records on a stream. -pub trait Inspect: InspectCore + Sized { +pub trait Inspect: InspectCore + Sized { /// Runs a supplied closure on each observed data element. /// /// # Examples @@ -90,14 +91,14 @@ pub trait Inspect: InspectCore + Sized { fn inspect_core(&self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; } -impl Inspect for StreamCore { +impl Inspect for StreamCore { fn inspect_core(&self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>) + 'static { self.inspect_container(func) } } /// Inspect containers -pub trait InspectCore { +pub trait InspectCore { /// Runs a supplied closure on each observed container, and each frontier advancement. /// /// Rust's `Result` type is used to distinguish the events, with `Ok` for time and data, @@ -120,7 +121,7 @@ pub trait InspectCore { fn inspect_container(&self, func: F) -> StreamCore where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; } -impl InspectCore for StreamCore { +impl InspectCore for StreamCore { fn inspect_container(&self, mut func: F) -> StreamCore where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static diff --git a/timely/src/dataflow/operators/core/map.rs b/timely/src/dataflow/operators/core/map.rs index 8af70e4a4..140eeba62 100644 --- a/timely/src/dataflow/operators/core/map.rs +++ b/timely/src/dataflow/operators/core/map.rs @@ -1,13 +1,13 @@ //! Extension methods for `StreamCore` based on record-by-record transformation. -use crate::container::{Container, SizableContainer, PushInto}; -use crate::Data; +use crate::container::{DrainContainer, SizableContainer, PushInto}; +use crate::Container; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::operator::Operator; /// Extension trait for `Stream`. -pub trait Map { +pub trait Map { /// Consumes each element of the stream and yields a new element. /// /// # Examples @@ -24,7 +24,7 @@ pub trait Map { /// ``` fn map(&self, mut logic: L) -> StreamCore where - C2: SizableContainer + PushInto + Data, + C2: Container + SizableContainer + PushInto, L: FnMut(C::Item<'_>)->D2 + 'static, { self.flat_map(move |x| std::iter::once(logic(x))) @@ -46,19 +46,19 @@ pub trait Map { fn flat_map(&self, logic: L) -> StreamCore where I: IntoIterator, - C2: SizableContainer + PushInto + Data, + C2: Container + SizableContainer + PushInto, L: FnMut(C::Item<'_>)->I + 'static, ; } -impl Map for StreamCore { +impl Map for StreamCore { // TODO : This would be more robust if it captured an iterator and then pulled an appropriate // TODO : number of elements from the iterator. This would allow iterators that produce many // TODO : records without taking arbitrarily long and arbitrarily much memory. fn flat_map(&self, mut logic: L) -> StreamCore where I: IntoIterator, - C2: SizableContainer + PushInto + Data, + C2: Container + SizableContainer + PushInto, L: FnMut(C::Item<'_>)->I + 'static, { self.unary(Pipeline, "FlatMap", move |_,_| move |input, output| { diff --git a/timely/src/dataflow/operators/core/ok_err.rs b/timely/src/dataflow/operators/core/ok_err.rs index 77fd1f527..339b51576 100644 --- a/timely/src/dataflow/operators/core/ok_err.rs +++ b/timely/src/dataflow/operators/core/ok_err.rs @@ -1,13 +1,13 @@ //! Operators that separate one stream into two streams based on some condition -use crate::container::{Container, SizableContainer, PushInto}; -use crate::Data; +use crate::Container; +use crate::container::{DrainContainer, SizableContainer, PushInto}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, StreamCore}; /// Extension trait for `Stream`. -pub trait OkErr { +pub trait OkErr { /// Takes one input stream and splits it into two output streams. /// For each record, the supplied closure is called with the data. /// If it returns `Ok(x)`, then `x` will be sent @@ -33,20 +33,20 @@ pub trait OkErr { logic: L, ) -> (StreamCore, StreamCore) where - C1: SizableContainer + PushInto + Data, - C2: SizableContainer + PushInto + Data, + C1: Container + SizableContainer + PushInto, + C2: Container + SizableContainer + PushInto, L: FnMut(C::Item<'_>) -> Result+'static ; } -impl OkErr for StreamCore { +impl OkErr for StreamCore { fn ok_err( &self, mut logic: L, ) -> (StreamCore, StreamCore) where - C1: SizableContainer + PushInto + Data, - C2: SizableContainer + PushInto + Data, + C1: Container + SizableContainer + PushInto, + C2: Container + SizableContainer + PushInto, L: FnMut(C::Item<'_>) -> Result+'static { let mut builder = OperatorBuilder::new("OkErr".to_owned(), self.scope()); diff --git a/timely/src/dataflow/operators/core/partition.rs b/timely/src/dataflow/operators/core/partition.rs index 58205364b..7602b4892 100644 --- a/timely/src/dataflow/operators/core/partition.rs +++ b/timely/src/dataflow/operators/core/partition.rs @@ -1,15 +1,14 @@ //! Partition a stream of records into multiple streams. -use timely_container::{Container, ContainerBuilder, PushInto}; - +use crate::container::{DrainContainer, ContainerBuilder, PushInto}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::operators::InputCapability; use crate::dataflow::{Scope, StreamCore}; -use crate::Data; +use crate::Container; /// Partition a stream of records into multiple streams. -pub trait Partition { +pub trait Partition { /// Produces `parts` output streams, containing records produced and assigned by `route`. /// /// # Examples @@ -31,15 +30,13 @@ pub trait Partition { fn partition(&self, parts: u64, route: F) -> Vec> where CB: ContainerBuilder + PushInto, - CB::Container: Data, F: FnMut(C::Item<'_>) -> (u64, D2) + 'static; } -impl Partition for StreamCore { +impl Partition for StreamCore { fn partition(&self, parts: u64, mut route: F) -> Vec> where CB: ContainerBuilder + PushInto, - CB::Container: Data, F: FnMut(C::Item<'_>) -> (u64, D2) + 'static, { let mut builder = OperatorBuilder::new("Partition".to_owned(), self.scope()); diff --git a/timely/src/dataflow/operators/core/probe.rs b/timely/src/dataflow/operators/core/probe.rs index cb3c96b34..324061991 100644 --- a/timely/src/dataflow/operators/core/probe.rs +++ b/timely/src/dataflow/operators/core/probe.rs @@ -13,7 +13,7 @@ use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; use crate::dataflow::{StreamCore, Scope}; -use crate::{Container, Data}; +use crate::Container; /// Monitors progress at a `Stream`. pub trait Probe { @@ -79,7 +79,7 @@ pub trait Probe { fn probe_with(&self, handle: &Handle) -> StreamCore; } -impl Probe for StreamCore { +impl Probe for StreamCore { fn probe(&self) -> Handle { // the frontier is shared state; scope updates, handle reads. diff --git a/timely/src/dataflow/operators/core/rc.rs b/timely/src/dataflow/operators/core/rc.rs index fdc68b9d4..e9c622bc7 100644 --- a/timely/src/dataflow/operators/core/rc.rs +++ b/timely/src/dataflow/operators/core/rc.rs @@ -3,11 +3,11 @@ use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::Operator; use crate::dataflow::{Scope, StreamCore}; -use crate::{Container, Data}; +use crate::Container; use std::rc::Rc; /// Convert a stream into a stream of shared containers -pub trait SharedStream { +pub trait SharedStream { /// Convert a stream into a stream of shared data /// /// # Examples @@ -24,7 +24,7 @@ pub trait SharedStream { fn shared(&self) -> StreamCore>; } -impl SharedStream for StreamCore { +impl SharedStream for StreamCore { fn shared(&self) -> StreamCore> { self.unary(Pipeline, "Shared", move |_, _| { move |input, output| { diff --git a/timely/src/dataflow/operators/core/reclock.rs b/timely/src/dataflow/operators/core/reclock.rs index e74bedbc4..86ec5ae69 100644 --- a/timely/src/dataflow/operators/core/reclock.rs +++ b/timely/src/dataflow/operators/core/reclock.rs @@ -1,6 +1,6 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::{Container, Data}; +use crate::Container; use crate::order::PartialOrder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; @@ -45,11 +45,11 @@ pub trait Reclock { /// assert_eq!(extracted[1], (5, vec![4,5])); /// assert_eq!(extracted[2], (8, vec![6,7,8])); /// ``` - fn reclock(&self, clock: &StreamCore) -> Self; + fn reclock(&self, clock: &StreamCore) -> Self; } -impl Reclock for StreamCore { - fn reclock(&self, clock: &StreamCore) -> StreamCore { +impl Reclock for StreamCore { + fn reclock(&self, clock: &StreamCore) -> StreamCore { let mut stash = vec![]; diff --git a/timely/src/dataflow/operators/core/to_stream.rs b/timely/src/dataflow/operators/core/to_stream.rs index cfc6f429b..4d2dea525 100644 --- a/timely/src/dataflow/operators/core/to_stream.rs +++ b/timely/src/dataflow/operators/core/to_stream.rs @@ -1,7 +1,7 @@ //! Conversion to the `StreamCore` type from iterators. use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, PushInto}; -use crate::{Container, Data}; +use crate::Container; use crate::dataflow::operators::generic::operator::source; use crate::dataflow::{StreamCore, Scope}; @@ -61,7 +61,7 @@ impl ToStreamBuilder for I wh /// Converts to a timely [StreamCore]. Equivalent to [`ToStreamBuilder`] but /// uses a [`CapacityContainerBuilder`]. -pub trait ToStream { +pub trait ToStream { /// Converts to a timely [StreamCore]. /// /// # Examples @@ -81,7 +81,7 @@ pub trait ToStream { fn to_stream(self, scope: &mut S) -> StreamCore; } -impl ToStream for I where C: PushInto { +impl ToStream for I where C: PushInto { fn to_stream(self, scope: &mut S) -> StreamCore { ToStreamBuilder::>::to_stream_with_builder(self, scope) } diff --git a/timely/src/dataflow/operators/core/unordered_input.rs b/timely/src/dataflow/operators/core/unordered_input.rs index 77c6ba81b..703a461d3 100644 --- a/timely/src/dataflow/operators/core/unordered_input.rs +++ b/timely/src/dataflow/operators/core/unordered_input.rs @@ -2,7 +2,8 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::{Container, Data}; + +use crate::Container; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; use crate::scheduling::{Schedule, ActivateOnDrop}; @@ -164,7 +165,7 @@ impl UnorderedHandle { } } -impl UnorderedHandle> { +impl UnorderedHandle> { /// Allocates a new automatically flushing session based on the supplied capability. #[inline] pub fn session(&mut self, cap: ActivateCapability) -> ActivateOnDrop, Counter>>> { diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index dc9db01ce..a10ff2f40 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -15,14 +15,14 @@ use crate::dataflow::channels::pushers::Counter as PushCounter; use crate::dataflow::channels::pushers::buffer::{Buffer, Session}; use crate::dataflow::channels::Message; use crate::communication::{Push, Pull}; -use crate::{Container, Data}; +use crate::{Container, Accountable}; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; use crate::dataflow::operators::InputCapability; use crate::dataflow::operators::capability::CapabilityTrait; /// Handle to an operator's input stream. -pub struct InputHandleCore>> { +pub struct InputHandleCore>> { pull_counter: PullCounter, internal: Rc>>>>>, /// Timestamp summaries from this input to each output. @@ -36,7 +36,7 @@ pub struct InputHandleCore>> { pub type InputHandle = InputHandleCore, P>; /// Handle to an operator's input stream and frontier. -pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: Container+'a, P: Pull>+'a> { +pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: 'a, P: Pull>+'a> { /// The underlying input handle. pub handle: &'a mut InputHandleCore, /// The frontier as reported by timely progress tracking. @@ -46,7 +46,7 @@ pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: Container+'a, P: Pull< /// Handle to an operator's input stream and frontier, specialized to vectors. pub type FrontieredInputHandle<'a, T, D, P> = FrontieredInputHandleCore<'a, T, Vec, P>; -impl>> InputHandleCore { +impl>> InputHandleCore { /// Reads the next input buffer (at some timestamp `t`) and a corresponding capability for `t`. /// The timestamp `t` of the input buffer can be retrieved by invoking `.time()` on the capability. @@ -87,7 +87,7 @@ impl>> InputHandleCore>+'a> FrontieredInputHandleCore<'a, T, C, P> { +impl<'a, T: Timestamp, C: Accountable, P: Pull>+'a> FrontieredInputHandleCore<'a, T, C, P> { /// Allocate a new frontiered input handle. pub fn new(handle: &'a mut InputHandleCore, frontier: &'a MutableAntichain) -> Self { FrontieredInputHandleCore { @@ -134,13 +134,13 @@ impl<'a, T: Timestamp, C: Container, P: Pull>+'a> FrontieredInputH } } -pub fn _access_pull_counter>>(input: &mut InputHandleCore) -> &mut PullCounter { +pub fn _access_pull_counter>>(input: &mut InputHandleCore) -> &mut PullCounter { &mut input.pull_counter } /// Constructs an input handle. /// Declared separately so that it can be kept private when `InputHandle` is re-exported. -pub fn new_input_handle>>( +pub fn new_input_handle>>( pull_counter: PullCounter, internal: Rc>>>>>, summaries: Rc>>, @@ -231,7 +231,7 @@ impl<'a, T: Timestamp, CB: ContainerBuilder, P: Push>> } } -impl<'a, T: Timestamp, C: Container + Data, P: Push>> OutputHandleCore<'a, T, CapacityContainerBuilder, P> { +impl<'a, T: Timestamp, C: Container, P: Push>> OutputHandleCore<'a, T, CapacityContainerBuilder, P> { /// Obtains a session that can send data at the timestamp associated with capability `cap`. /// /// In order to send data at a future timestamp, obtain a capability for the new timestamp diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index 5f76bc59c..79f6076e4 100644 --- a/timely/src/dataflow/operators/generic/operator.rs +++ b/timely/src/dataflow/operators/generic/operator.rs @@ -12,11 +12,11 @@ use crate::dataflow::{Scope, StreamCore}; use super::builder_rc::OperatorBuilder; use crate::dataflow::operators::generic::OperatorInfo; use crate::dataflow::operators::generic::notificator::{Notificator, FrontierNotificator}; -use crate::{Container, Data}; +use crate::Container; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; /// Methods to construct generic streaming and blocking operators. -pub trait Operator { +pub trait Operator { /// Creates a new dataflow operator that partitions its input stream by a parallelization /// strategy `pact`, and repeatedly invokes `logic`, the function returned by the function passed as `constructor`. /// `logic` can read from the input stream, write to the output stream, and inspect the frontier at the input. @@ -177,7 +177,7 @@ pub trait Operator { /// ``` fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, @@ -227,7 +227,7 @@ pub trait Operator { /// } /// }).unwrap(); /// ``` - fn binary_notify, &mut InputHandleCore, @@ -269,7 +269,7 @@ pub trait Operator { /// ``` fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, @@ -307,7 +307,7 @@ pub trait Operator { P: ParallelizationContract; } -impl Operator for StreamCore { +impl Operator for StreamCore { fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> StreamCore where @@ -388,7 +388,7 @@ impl Operator for StreamCore { fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, @@ -419,7 +419,7 @@ impl Operator for StreamCore { stream } - fn binary_notify, &mut InputHandleCore, @@ -447,7 +447,7 @@ impl Operator for StreamCore { fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, @@ -580,7 +580,7 @@ where /// /// }); /// ``` -pub fn empty(scope: &G) -> StreamCore { +pub fn empty(scope: &G) -> StreamCore { source::<_, CapacityContainerBuilder, _, _>(scope, "Empty", |_capability, _info| |_output| { // drop capability, do nothing }) diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index 7959c842a..b5bb93a01 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -11,7 +11,6 @@ use crate::dataflow::Scope; use crate::dataflow::channels::pushers::tee::TeeHelper; use crate::dataflow::channels::Message; use std::fmt::{self, Debug}; -use crate::Container; // use dataflow::scopes::root::loggers::CHANNELS_Q; @@ -47,7 +46,7 @@ impl Clone for StreamCore { /// A stream batching data in vectors. pub type Stream = StreamCore>; -impl StreamCore { +impl StreamCore { /// Connects the stream to a destination. /// /// The destination is described both by a `Target`, for progress tracking information, and a `P: Push` where the @@ -76,7 +75,7 @@ impl StreamCore { pub fn scope(&self) -> S { self.scope.clone() } /// Allows the assertion of a container type, for the benefit of type inference. - pub fn container(self) -> StreamCore where Self: AsStream { self.as_stream() } + pub fn container(self) -> StreamCore where Self: AsStream { self.as_stream() } } /// A type that can be translated to a [StreamCore]. diff --git a/timely/src/lib.rs b/timely/src/lib.rs index 557b7c7e1..20f3f995e 100644 --- a/timely/src/lib.rs +++ b/timely/src/lib.rs @@ -65,7 +65,7 @@ pub use timely_communication::Config as CommunicationConfig; pub use worker::Config as WorkerConfig; pub use execute::Config as Config; -pub use timely_container::Container; +pub use timely_container::Accountable; /// Re-export of the `timely_container` crate. pub mod container { pub use timely_container::*; @@ -104,6 +104,12 @@ pub mod scheduling; pub trait Data: Clone+'static { } impl Data for T { } +/// A composite trait for types usable as containers in timely dataflow. +/// +/// The `Container` trait is necessary for all containers in timely dataflow channels. +pub trait Container: Accountable + Default + Clone + 'static { } +impl Container for C { } + /// A composite trait for types usable on exchange channels in timely dataflow. /// /// The `ExchangeData` trait extends `Data` with any requirements imposed by the `timely_communication` diff --git a/timely/src/logging.rs b/timely/src/logging.rs index 2c2439319..9122f6da4 100644 --- a/timely/src/logging.rs +++ b/timely/src/logging.rs @@ -135,7 +135,7 @@ pub struct MessagesEvent { /// Message sequence number. pub seq_no: usize, /// Number of typed records in the message. - pub length: usize, + pub record_count: i64, } /// Records the starting and stopping of an operator.