From 0d03378d7088a3dba226b8504257d4515d14247b Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Tue, 22 Jul 2025 09:19:27 +0200 Subject: [PATCH 1/7] Progress container with length Splits the Container trait into a progress container and a container. The progress container only exposes a length, which is enough for most parts of Timely. The container trait extends the progress container and carries the usual functions to iterate and clear the contents. I'm open to renaming and moving types, and primarily wanted to check whether it's possible to split the trait. Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 124 +++++++++++------- timely/examples/columnar.rs | 5 +- timely/src/dataflow/channels/mod.rs | 8 +- timely/src/dataflow/channels/pact.rs | 12 +- .../src/dataflow/channels/pullers/counter.rs | 4 +- .../src/dataflow/channels/pushers/buffer.rs | 2 +- .../src/dataflow/channels/pushers/counter.rs | 4 +- .../src/dataflow/channels/pushers/exchange.rs | 6 +- timely/src/dataflow/channels/pushers/tee.rs | 10 +- timely/src/dataflow/operators/branch.rs | 13 +- .../operators/core/capture/capture.rs | 7 +- .../operators/core/capture/extract.rs | 5 +- timely/src/dataflow/operators/core/concat.rs | 18 +-- .../src/dataflow/operators/core/enterleave.rs | 19 +-- .../src/dataflow/operators/core/feedback.rs | 22 ++-- timely/src/dataflow/operators/core/input.rs | 3 +- timely/src/dataflow/operators/core/inspect.rs | 9 +- timely/src/dataflow/operators/core/probe.rs | 13 +- timely/src/dataflow/operators/core/rc.rs | 14 +- timely/src/dataflow/operators/core/reclock.rs | 13 +- .../src/dataflow/operators/core/to_stream.rs | 4 +- .../dataflow/operators/generic/builder_raw.rs | 10 +- .../dataflow/operators/generic/builder_rc.rs | 6 +- .../src/dataflow/operators/generic/handles.rs | 14 +- .../dataflow/operators/generic/operator.rs | 32 +++-- timely/src/dataflow/stream.rs | 13 +- 26 files changed, 217 insertions(+), 173 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 839f80167..0736c9990 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -7,27 +7,11 @@ use std::collections::VecDeque; /// A container transferring data through dataflow edges /// /// 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`). +/// whether it is empty (`is_empty()`). /// /// 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) - } - +pub trait ProgressContainer: Default { /// The number of elements in this container /// /// This number is used in progress tracking to confirm the receipt of some number @@ -37,9 +21,27 @@ pub trait Container: Default { fn len(&self) -> usize; /// Determine if the container contains any elements, corresponding to `len() == 0`. + #[inline(always)] fn is_empty(&self) -> bool { self.len() == 0 } +} + +/// A container that can reveal its contents through iterating by reference and draining. +/// +/// It supports removing all elements (`clear`). +pub trait Container: ProgressContainer { + /// 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) + } /// Remove all contents from `self` while retaining allocated memory. /// After calling `clear`, `is_empty` must return `true` and `len` 0. @@ -102,7 +104,7 @@ 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; + type Container: ProgressContainer + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -118,6 +120,7 @@ pub trait ContainerBuilder: Default + 'static { where Self: for<'a> PushInto<::Item<'a>>, I: for<'a> FnMut(&::Item<'a>) -> usize, + Self::Container: Container, { for datum in container.drain() { let index = index(&datum); @@ -142,6 +145,35 @@ pub trait ContainerBuilder: Default + 'static { /// If you have any questions about this trait you are best off not implementing it. pub trait LengthPreservingContainerBuilder : ContainerBuilder { } +/// A container builder that never produces any outputs, and can be used to pass through data in +/// operators. +#[derive(Debug, Clone)] +pub struct PassthroughContainerBuilder(std::marker::PhantomData); + +impl Default for PassthroughContainerBuilder { + #[inline(always)] + fn default() -> Self { + PassthroughContainerBuilder(std::marker::PhantomData) + } +} + +impl ContainerBuilder for PassthroughContainerBuilder +where + C: ProgressContainer + Clone + 'static, +{ + type Container = C; + + #[inline(always)] + fn extract(&mut self) -> Option<&mut Self::Container> { + None + } + + #[inline(always)] + fn finish(&mut self) -> Option<&mut Self::Container> { + None + } +} + /// A default container builder that uses length and preferred capacity to chunk data. /// /// Maintains a single empty allocation between [`Self::push_into`] and [`Self::extract`], but not @@ -165,7 +197,7 @@ impl> PushInto for CapacityContainerBuil 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() { @@ -199,18 +231,18 @@ impl ContainerBuilder for CapacityContainerBuild impl LengthPreservingContainerBuilder for CapacityContainerBuilder { } +impl ProgressContainer for Vec { + #[inline(always)] + fn len(&self) -> usize { Vec::len(self) } + + #[inline(always)] + fn is_empty(&self) -> bool { Vec::is_empty(self) } +} + 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) } type Iter<'a> = std::slice::Iter<'a, T> where Self: 'a; @@ -268,20 +300,20 @@ mod rc { use std::ops::Deref; use std::rc::Rc; - use crate::Container; + use crate::{Container, ProgressContainer}; + + impl ProgressContainer for Rc { + #[inline(always)] + fn len(&self) -> usize { std::ops::Deref::deref(self).len() } + + #[inline(always)] + fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } + } impl Container 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) { @@ -309,20 +341,20 @@ mod arc { use std::ops::Deref; use std::sync::Arc; - use crate::Container; + use crate::{Container, ProgressContainer}; + + impl ProgressContainer for Arc { + #[inline(always)] + fn len(&self) -> usize { std::ops::Deref::deref(self).len() } + + #[inline(always)] + fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } + } impl Container 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) { diff --git a/timely/examples/columnar.rs b/timely/examples/columnar.rs index d1869ee0a..3c03c677f 100644 --- a/timely/examples/columnar.rs +++ b/timely/examples/columnar.rs @@ -165,8 +165,11 @@ mod container { } } - impl timely::Container for Column { + impl timely::container::ProgressContainer for Column { + #[inline(always)] fn len(&self) -> usize { self.borrow().len() } + } + impl timely::Container for Column { // This sets `self` to be an empty `Typed` variant, appropriate for pushing into. fn clear(&mut self) { match self { diff --git a/timely/src/dataflow/channels/mod.rs b/timely/src/dataflow/channels/mod.rs index 6a351753b..2f4274508 100644 --- a/timely/src/dataflow/channels/mod.rs +++ b/timely/src/dataflow/channels/mod.rs @@ -2,7 +2,7 @@ use serde::{Deserialize, Serialize}; use crate::communication::Push; -use crate::Container; +use crate::container::ProgressContainer; /// A collection of types that may be pushed at. pub mod pushers; @@ -32,14 +32,15 @@ impl Message { } } -impl Message { +impl Message { /// Creates a new message instance from arguments. pub fn new(time: T, data: C, from: usize, seq: usize) -> Self { Message { time, data, from, seq } } /// 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's contents are left in an + /// undefined state, specifically the caller cannot rely on this function clearing the buffer. #[inline] pub fn push_at>>(buffer: &mut C, time: T, pusher: &mut P) { @@ -51,7 +52,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 55404015c..a401c88dd 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -10,7 +10,7 @@ use std::{fmt::{self, Debug}, marker::PhantomData}; use std::rc::Rc; -use crate::{Container, container::{ContainerBuilder, LengthPreservingContainerBuilder, SizableContainer, CapacityContainerBuilder, PushInto}}; +use crate::{Container, container::{ContainerBuilder, LengthPreservingContainerBuilder, ProgressContainer, SizableContainer, CapacityContainerBuilder, PushInto}}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; use crate::communication::{Push, Pull}; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; @@ -34,7 +34,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) { @@ -52,7 +52,7 @@ pub type Exchange = ExchangeCore>, F>; impl ExchangeCore where - CB: LengthPreservingContainerBuilder, + CB: LengthPreservingContainerBuilder, for<'a> F: FnMut(&::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. @@ -81,7 +81,7 @@ 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, + CB: ContainerBuilder, CB: for<'a> PushInto<::Item<'a>>, CB::Container: Data + Send + crate::dataflow::channels::ContainerBytes, for<'a> H: FnMut(&::Item<'a>) -> u64 @@ -129,7 +129,7 @@ impl>> LogPusher { } } -impl>> Push> for LogPusher { +impl>> Push> for LogPusher { #[inline] fn push(&mut self, pair: &mut Option>) { if let Some(bundle) = pair { @@ -179,7 +179,7 @@ impl>> LogPuller { } } -impl>> Pull> for LogPuller { +impl>> Pull> for LogPuller { #[inline] fn pull(&mut self) -> &mut Option> { let result = self.puller.pull(); diff --git a/timely/src/dataflow/channels/pullers/counter.rs b/timely/src/dataflow/channels/pullers/counter.rs index 6f71848a4..ec5070630 100644 --- a/timely/src/dataflow/channels/pullers/counter.rs +++ b/timely/src/dataflow/channels/pullers/counter.rs @@ -6,7 +6,7 @@ use std::cell::RefCell; use crate::dataflow::channels::Message; use crate::progress::ChangeBatch; use crate::communication::Pull; -use crate::Container; +use crate::container::ProgressContainer; /// A wrapper which accounts records pulled past in a shared count map. pub struct Counter>> { @@ -36,7 +36,7 @@ impl Drop for ConsumedGuard { } } -impl>> Counter { +impl>> Counter { /// Retrieves the next timestamp and batch of data. #[inline] pub fn next(&mut self) -> Option<&mut Message> { diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index be579936c..93e05754a 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,7 +2,7 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, ProgressContainer, PushInto}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; diff --git a/timely/src/dataflow/channels/pushers/counter.rs b/timely/src/dataflow/channels/pushers/counter.rs index a6a16ce5d..1048e8d5c 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::container::ProgressContainer; /// A wrapper which updates shared `produced` based on the number of records pushed. #[derive(Debug)] @@ -17,7 +17,7 @@ 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 { diff --git a/timely/src/dataflow/channels/pushers/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index fec26cae7..0d145f8d2 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -9,7 +9,7 @@ use crate::{Container, Data}; /// Distributes records among target pushees according to a distribution function. pub struct Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, for<'a> H: FnMut(&::Item<'a>) -> u64 { @@ -21,7 +21,7 @@ where impl Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, for<'a> H: FnMut(&::Item<'a>) -> u64 { @@ -50,7 +50,7 @@ where impl Push> for Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, CB: for<'a> PushInto<::Item<'a>>, P: Push>, for<'a> H: FnMut(&::Item<'a>) -> u64 diff --git a/timely/src/dataflow/channels/pushers/tee.rs b/timely/src/dataflow/channels/pushers/tee.rs index 49d9e17d0..ff6ff9daa 100644 --- a/timely/src/dataflow/channels/pushers/tee.rs +++ b/timely/src/dataflow/channels/pushers/tee.rs @@ -5,9 +5,9 @@ use std::fmt::{self, Debug}; use std::rc::Rc; use crate::dataflow::channels::Message; - use crate::communication::Push; -use crate::{Container, Data}; +use crate::Data; +use crate::container::ProgressContainer; type PushList = Rc>>>>>; @@ -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(); @@ -39,7 +39,7 @@ impl Push> for Tee { } } -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..2cafaa403 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -1,9 +1,10 @@ //! Operators that separate one stream into two streams based on some condition +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, Stream, StreamCore}; -use crate::{Container, Data}; +use crate::Data; /// Extension trait for `Stream`. pub trait Branch { @@ -93,14 +94,14 @@ 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); let mut input = builder.new_input(self, Pipeline); - let (mut output1, stream1) = builder.new_output(); - let (mut output2, stream2) = builder.new_output(); + let (mut output1, stream1) = builder.new_output::>(); + let (mut output2, stream2) = builder.new_output::>(); builder.build(move |_| { @@ -110,9 +111,9 @@ impl BranchWhen for StreamCore { +pub trait Capture { /// Captures a stream of timestamped data for later replay. /// /// # Examples @@ -117,7 +118,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..e311dae85 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::PushInto; +use crate::Container; /// Supports extracting a sequence of timestamp and data. pub trait Extract { @@ -48,7 +49,7 @@ pub trait Extract { fn extract(self) -> Vec<(T, C)>; } -impl Extract for ::std::sync::mpsc::Receiver> +impl Extract for ::std::sync::mpsc::Receiver> where for<'a> C: PushInto>, for<'a> C::Item<'a>: Ord, diff --git a/timely/src/dataflow/operators/core/concat.rs b/timely/src/dataflow/operators/core/concat.rs index 24fcef532..de3d36c2f 100644 --- a/timely/src/dataflow/operators/core/concat.rs +++ b/timely/src/dataflow/operators/core/concat.rs @@ -1,12 +1,12 @@ //! Merges the contents of multiple streams. - -use crate::{Container, Data}; +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::Data; 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 +23,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 +52,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 +62,7 @@ impl Concatenate for StreamCore { } } -impl Concatenate for G { +impl Concatenate for G { fn concatenate(&self, sources: I) -> StreamCore where I: IntoIterator> @@ -77,7 +77,7 @@ impl Concatenate for G { let mut handles = sources.into_iter().map(|s| builder.new_input(&s, Pipeline)).collect::>(); // create one output handle for the concatenated results. - let (mut output, result) = builder.new_output(); + let (mut output, result) = builder.new_output::>(); // build an operator that plays out all input data. builder.build(move |_capability| { @@ -86,7 +86,7 @@ impl Concatenate for G { let mut output = output.activate(); for handle in handles.iter_mut() { handle.for_each(|time, data| { - output.session(&time).give_container(data); + output.session_with_builder(&time).give_container(data); }) } } diff --git a/timely/src/dataflow/operators/core/enterleave.rs b/timely/src/dataflow/operators/core/enterleave.rs index bc221c4a6..b0178d0ae 100644 --- a/timely/src/dataflow/operators/core/enterleave.rs +++ b/timely/src/dataflow/operators/core/enterleave.rs @@ -22,11 +22,12 @@ use std::marker::PhantomData; use std::rc::Rc; +use crate::container::ProgressContainer; 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::{Data}; use crate::communication::Push; use crate::dataflow::channels::pushers::{Counter, Tee}; use crate::dataflow::channels::Message; @@ -35,7 +36,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 +54,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: Data+ProgressContainer> Enter for StreamCore { fn enter<'a>(&self, scope: &Child<'a, G, T>) -> StreamCore, C> { use crate::scheduling::Scheduler; @@ -85,7 +86,7 @@ impl, C: Data+Container> Enter { +pub trait Leave { /// Moves a `Stream` to the parent of its current `Scope`. /// /// # Examples @@ -103,7 +104,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 +131,14 @@ impl> Leave, TContainer: Container + Data> { +struct IngressNub, TContainer: ProgressContainer + Data> { targets: Counter>, phantom: ::std::marker::PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TContainer: Container + Data> Push> for IngressNub { +impl, TContainer: ProgressContainer + Data> 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); @@ -164,7 +165,7 @@ struct EgressNub, TContaine phantom: PhantomData, } -impl Push> for EgressNub +impl Push> for EgressNub where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: Data { fn push(&mut self, message: &mut Option>) { if let Some(inner_message) = message { @@ -207,7 +208,7 @@ impl

LogPusher

{ impl Push> for LogPusher

where - C: Container, + C: ProgressContainer, P: Push>, { fn push(&mut self, element: &mut Option>) { diff --git a/timely/src/dataflow/operators/core/feedback.rs b/timely/src/dataflow/operators/core/feedback.rs index 61bd5c196..779e7b3d1 100644 --- a/timely/src/dataflow/operators/core/feedback.rs +++ b/timely/src/dataflow/operators/core/feedback.rs @@ -1,7 +1,7 @@ //! Create cycles in a timely dataflow graph. -use crate::{Container, Data}; -use crate::container::CapacityContainerBuilder; +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::operators::generic::OutputWrapper; @@ -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; @@ -122,7 +122,7 @@ impl ConnectLoop for StreamCore { if let Some(new_time) = summary.results_in(cap.time()) { let new_cap = cap.delayed(&new_time); output - .session(&new_cap) + .session_with_builder(&new_cap) .give_container(data); } }); @@ -132,8 +132,8 @@ 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>, + output: OutputWrapper, Tee>, } diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index c5f974432..a0259c851 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,7 +3,7 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, ProgressContainer, PushInto}; use crate::scheduling::{Schedule, Activator}; @@ -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..5a850e29d 100644 --- a/timely/src/dataflow/operators/core/inspect.rs +++ b/timely/src/dataflow/operators/core/inspect.rs @@ -1,5 +1,6 @@ //! Extension trait and implementation for observing and action on streamed data. +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; use crate::{Container, Data}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; @@ -97,7 +98,7 @@ impl Inspect for StreamCore { } /// 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,14 +121,14 @@ 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 { use crate::progress::timestamp::Timestamp; let mut frontier = crate::progress::Antichain::from_elem(G::Timestamp::minimum()); - self.unary_frontier(Pipeline, "InspectBatch", move |_,_| move |input, output| { + self.unary_frontier::,_,_,_>(Pipeline, "InspectBatch", move |_,_| move |input, output| { if input.frontier.frontier() != frontier.borrow() { frontier.clear(); frontier.extend(input.frontier.frontier().iter().cloned()); @@ -135,7 +136,7 @@ impl InspectCore for StreamCore { } input.for_each(|time, data| { func(Ok((&time, &*data))); - output.session(&time).give_container(data); + output.session_with_builder(&time).give_container(data); }); }) } diff --git a/timely/src/dataflow/operators/core/probe.rs b/timely/src/dataflow/operators/core/probe.rs index cb3c96b34..2d00d6d73 100644 --- a/timely/src/dataflow/operators/core/probe.rs +++ b/timely/src/dataflow/operators/core/probe.rs @@ -10,13 +10,12 @@ use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; - - use crate::dataflow::{StreamCore, Scope}; -use crate::{Container, Data}; +use crate::Data; +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; /// Monitors progress at a `Stream`. -pub trait Probe { +pub trait Probe { /// Constructs a progress probe which indicates which timestamps have elapsed at the operator. /// /// # Examples @@ -79,7 +78,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. @@ -92,7 +91,7 @@ impl Probe for StreamCore { let mut builder = OperatorBuilder::new("Probe".to_owned(), self.scope()); let mut input = PullCounter::new(builder.new_input(self, Pipeline)); let (tee, stream) = builder.new_output(); - let mut output = PushBuffer::new(PushCounter::new(tee)); + let mut output = PushBuffer::<_, PassthroughContainerBuilder<_>, _>::new(PushCounter::new(tee)); let shared_frontier = Rc::downgrade(&handle.frontier); let mut started = false; @@ -115,7 +114,7 @@ impl Probe for StreamCore { while let Some(message) = input.next() { let time = &message.time; let data = &mut message.data; - output.session(time).give_container(data); + output.session_with_builder(time).give_container(data); } output.cease(); diff --git a/timely/src/dataflow/operators/core/rc.rs b/timely/src/dataflow/operators/core/rc.rs index fdc68b9d4..ac03b61f1 100644 --- a/timely/src/dataflow/operators/core/rc.rs +++ b/timely/src/dataflow/operators/core/rc.rs @@ -1,13 +1,15 @@ //! Shared containers +use std::rc::Rc; + +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::Operator; use crate::dataflow::{Scope, StreamCore}; -use crate::{Container, Data}; -use std::rc::Rc; +use crate::Data; /// 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,13 +26,13 @@ pub trait SharedStream { fn shared(&self) -> StreamCore>; } -impl SharedStream for StreamCore { +impl SharedStream for StreamCore { fn shared(&self) -> StreamCore> { - self.unary(Pipeline, "Shared", move |_, _| { + self.unary::,_,_,_>(Pipeline, "Shared", move |_, _| { move |input, output| { input.for_each(|time, data| { output - .session(&time) + .session_with_builder(&time) .give_container(&mut Rc::new(std::mem::take(data))); }); } diff --git a/timely/src/dataflow/operators/core/reclock.rs b/timely/src/dataflow/operators/core/reclock.rs index e74bedbc4..e82333800 100644 --- a/timely/src/dataflow/operators/core/reclock.rs +++ b/timely/src/dataflow/operators/core/reclock.rs @@ -1,6 +1,7 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::{Container, Data}; +use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::Data; use crate::order::PartialOrder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; @@ -45,15 +46,15 @@ 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![]; - self.binary_notify(clock, Pipeline, Pipeline, "Reclock", vec![], move |input1, input2, output, notificator| { + self.binary_notify::<_,PassthroughContainerBuilder<_>,_,_,_>(clock, Pipeline, Pipeline, "Reclock", vec![], move |input1, input2, output, notificator| { // stash each data input with its timestamp. input1.for_each(|cap, data| { @@ -67,7 +68,7 @@ impl Reclock for StreamCore { // each time with complete stash can be flushed. notificator.for_each(|cap,_,_| { - let mut session = output.session(&cap); + let mut session = output.session_with_builder(&cap); for &mut (ref t, ref mut data) in &mut stash { if t.less_equal(cap.time()) { session.give_container(data); diff --git a/timely/src/dataflow/operators/core/to_stream.rs b/timely/src/dataflow/operators/core/to_stream.rs index cfc6f429b..3b340ba65 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::Data; 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 diff --git a/timely/src/dataflow/operators/generic/builder_raw.rs b/timely/src/dataflow/operators/generic/builder_raw.rs index 4659383a9..3ecf99131 100644 --- a/timely/src/dataflow/operators/generic/builder_raw.rs +++ b/timely/src/dataflow/operators/generic/builder_raw.rs @@ -10,10 +10,10 @@ use std::cell::RefCell; use crate::scheduling::{Schedule, Activations}; +use crate::container::ProgressContainer; use crate::progress::{Source, Target}; use crate::progress::{Timestamp, Operate, operate::SharedProgress, Antichain}; use crate::progress::operate::{Connectivity, PortConnectivity}; -use crate::Container; use crate::dataflow::{StreamCore, Scope}; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pact::ParallelizationContract; @@ -104,7 +104,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller where P: ParallelizationContract { @@ -113,7 +113,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller where P: ParallelizationContract, I: IntoIterator::Summary>)>, @@ -133,14 +133,14 @@ impl OperatorBuilder { } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (Tee, StreamCore) { + pub fn new_output(&mut self) -> (Tee, StreamCore) { let connection = (0 .. self.shape.inputs).map(|i| (i, Antichain::from_elem(Default::default()))); self.new_output_connection(connection) } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) + pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) where I: IntoIterator::Summary>)>, { diff --git a/timely/src/dataflow/operators/generic/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index 66957ddf5..4758fbed3 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -9,7 +9,7 @@ use crate::progress::operate::SharedProgress; use crate::progress::frontier::{Antichain, MutableAntichain}; use crate::Container; -use crate::container::ContainerBuilder; +use crate::container::{ContainerBuilder, ProgressContainer}; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pushers::Counter as PushCounter; @@ -60,7 +60,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore where P: ParallelizationContract { @@ -76,7 +76,7 @@ impl OperatorBuilder { /// /// Commonly the connections are either the unit summary, indicating the same timestamp might be produced as output, or an empty /// antichain indicating that there is no connection from the input to the output. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore where P: ParallelizationContract, I: IntoIterator::Summary>)> + Clone, diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index 6a289e95e..eb7e7b581 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -16,14 +16,14 @@ 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::{ContainerBuilder, CapacityContainerBuilder}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, ProgressContainer}; use crate::logging::TimelyLogger as Logger; 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. @@ -38,7 +38,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: ProgressContainer+'a, P: Pull>+'a> { /// The underlying input handle. pub handle: &'a mut InputHandleCore, /// The frontier as reported by timely progress tracking. @@ -48,7 +48,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. @@ -93,7 +93,7 @@ impl>> InputHandleCore>+'a> FrontieredInputHandleCore<'a, T, C, P> { +impl<'a, T: Timestamp, C: ProgressContainer, 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 { @@ -140,13 +140,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>>, diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index 1c51f7ecd..deb92d88c 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::{ContainerBuilder, CapacityContainerBuilder}; +use crate::Data; +use crate::container::{ContainerBuilder, PassthroughContainerBuilder, ProgressContainer}; /// 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: ProgressContainer + Data, 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: ProgressContainer + Data, 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 @@ -389,7 +389,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: ProgressContainer + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, @@ -420,7 +420,7 @@ impl Operator for StreamCore { stream } - fn binary_notify, &mut InputHandleCore, @@ -449,7 +449,7 @@ impl Operator for StreamCore { fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: ProgressContainer + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, @@ -582,8 +582,12 @@ where /// /// }); /// ``` -pub fn empty(scope: &G) -> StreamCore { - source::<_, CapacityContainerBuilder, _, _>(scope, "Empty", |_capability, _info| |_output| { - // drop capability, do nothing - }) +pub fn empty(scope: &G) -> StreamCore { + let mut builder = OperatorBuilder::new("Empty".to_owned(), scope.clone()); + + let (_output, stream) = builder.new_output::>(); + builder.set_notify(false); + builder.build(|_caps| |_frontier| { }); + + stream } diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index 7959c842a..0d1e14180 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -4,18 +4,17 @@ //! operator output. Extension methods on the `Stream` type provide the appearance of higher-level //! declarative programming, while constructing a dataflow graph underneath. +use std::fmt::{self, Debug}; + use crate::progress::{Source, Target}; use crate::communication::Push; +use crate::container::ProgressContainer; 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; -/// Abstraction of a stream of `C: Container` records timestamped with `S::Timestamp`. +/// Abstraction of a stream of `C: ProgressContainer` records timestamped with `S::Timestamp`. /// /// Internally `Stream` maintains a list of data recipients who should be presented with data /// produced by the source of the stream. @@ -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]. From be571f3fc2bac4973609bfd1d42b6869946c400a Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Tue, 22 Jul 2025 11:41:40 +0200 Subject: [PATCH 2/7] Back out push_into change Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 0736c9990..38dbb3e03 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -197,7 +197,7 @@ impl> PushInto for CapacityContainerBuil self.current.ensure_capacity(&mut self.empty); // Push item - self.current.push_into(item); + self.current.push(item); // Maybe flush if self.current.at_capacity() { From ba7abf7efb0f6982cb7ae4e62e7fc1137c50dfd0 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 24 Jul 2025 10:21:26 +0200 Subject: [PATCH 3/7] Rename ProgressContainer to WithProgress Removes clear from Container Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 105 ++++++------------ timely/examples/columnar.rs | 17 +-- timely/src/dataflow/channels/mod.rs | 4 +- timely/src/dataflow/channels/pact.rs | 14 +-- .../src/dataflow/channels/pullers/counter.rs | 6 +- .../src/dataflow/channels/pushers/buffer.rs | 2 +- .../src/dataflow/channels/pushers/counter.rs | 6 +- timely/src/dataflow/channels/pushers/tee.rs | 8 +- timely/src/dataflow/operators/branch.rs | 4 +- .../operators/core/capture/capture.rs | 6 +- timely/src/dataflow/operators/core/concat.rs | 10 +- .../src/dataflow/operators/core/enterleave.rs | 18 +-- .../src/dataflow/operators/core/feedback.rs | 16 +-- timely/src/dataflow/operators/core/input.rs | 2 +- timely/src/dataflow/operators/core/inspect.rs | 4 +- timely/src/dataflow/operators/core/probe.rs | 4 +- timely/src/dataflow/operators/core/rc.rs | 4 +- timely/src/dataflow/operators/core/reclock.rs | 8 +- .../dataflow/operators/generic/builder_raw.rs | 10 +- .../dataflow/operators/generic/builder_rc.rs | 6 +- .../src/dataflow/operators/generic/handles.rs | 14 +-- .../dataflow/operators/generic/operator.rs | 20 ++-- timely/src/dataflow/stream.rs | 6 +- 23 files changed, 121 insertions(+), 173 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 38dbb3e03..627200c9a 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -4,33 +4,29 @@ use std::collections::VecDeque; -/// A container transferring data through dataflow edges +/// A type representing progress, with an update count. /// -/// A container stores a number of elements and thus is able to describe it length (`len()`) and -/// whether it is empty (`is_empty()`). +/// It describes its update count (`count()`) and whether it is empty (`is_empty()`). /// -/// A container must implement default. The default implementation is not required to allocate -/// memory for variable-length components. -pub trait ProgressContainer: Default { +/// We require [`Default`] for convenience purposes. +pub trait WithProgress: Default { /// The number of elements in this container /// /// 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 + /// imposed on the [`CountPreservingContainerBuilder`] trait, whose implementors /// must preserve the number of items. - fn len(&self) -> usize; + fn count(&self) -> usize; - /// Determine if the container contains any elements, corresponding to `len() == 0`. + /// Determine if the container contains any elements, corresponding to `count() == 0`. #[inline(always)] fn is_empty(&self) -> bool { - self.len() == 0 + self.count() == 0 } } /// A container that can reveal its contents through iterating by reference and draining. -/// -/// It supports removing all elements (`clear`). -pub trait Container: ProgressContainer { +pub trait Container: WithProgress { /// The type of elements when reading non-destructively from the container. type ItemRef<'a> where Self: 'a; @@ -43,10 +39,6 @@ pub trait Container: ProgressContainer { self.push_into(item) } - /// Remove all contents from `self` while retaining allocated memory. - /// After calling `clear`, `is_empty` must return `true` and `len` 0. - fn clear(&mut self); - /// Iterator type when reading from the container. type Iter<'a>: Iterator> where Self: 'a; @@ -104,7 +96,7 @@ 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: ProgressContainer + Clone + 'static; + type Container: WithProgress + Default + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -126,7 +118,6 @@ pub trait ContainerBuilder: Default + 'static { let index = index(&datum); builders[index].push_into(datum); } - container.clear(); } /// Indicates a good moment to release resources. @@ -143,7 +134,7 @@ pub trait ContainerBuilder: Default + 'static { /// 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. /// If you have any questions about this trait you are best off not implementing it. -pub trait LengthPreservingContainerBuilder : ContainerBuilder { } +pub trait CountPreservingContainerBuilder: ContainerBuilder { } /// A container builder that never produces any outputs, and can be used to pass through data in /// operators. @@ -157,9 +148,7 @@ impl Default for PassthroughContainerBuilder { } } -impl ContainerBuilder for PassthroughContainerBuilder -where - C: ProgressContainer + Clone + 'static, +impl ContainerBuilder for PassthroughContainerBuilder { type Container = C; @@ -206,7 +195,7 @@ impl> PushInto for CapacityContainerBuil } } -impl ContainerBuilder for CapacityContainerBuilder { +impl ContainerBuilder for CapacityContainerBuilder { type Container = C; #[inline] @@ -229,22 +218,11 @@ impl ContainerBuilder for CapacityContainerBuild } } -impl LengthPreservingContainerBuilder for CapacityContainerBuilder { } - -impl ProgressContainer for Vec { - #[inline(always)] - fn len(&self) -> usize { Vec::len(self) } - - #[inline(always)] - fn is_empty(&self) -> bool { Vec::is_empty(self) } -} +impl CountPreservingContainerBuilder for CapacityContainerBuilder { } impl Container for Vec { type ItemRef<'a> = &'a T where T: 'a; type Item<'a> = T where T: 'a; - - fn clear(&mut self) { Vec::clear(self) } - type Iter<'a> = std::slice::Iter<'a, T> where Self: 'a; fn iter(&self) -> Self::Iter<'_> { @@ -300,29 +278,11 @@ mod rc { use std::ops::Deref; use std::rc::Rc; - use crate::{Container, ProgressContainer}; - - impl ProgressContainer for Rc { - #[inline(always)] - fn len(&self) -> usize { std::ops::Deref::deref(self).len() } - - #[inline(always)] - fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } - } + use crate::Container; impl Container for Rc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; - - 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<'_> { @@ -341,29 +301,11 @@ mod arc { use std::ops::Deref; use std::sync::Arc; - use crate::{Container, ProgressContainer}; - - impl ProgressContainer for Arc { - #[inline(always)] - fn len(&self) -> usize { std::ops::Deref::deref(self).len() } - - #[inline(always)] - fn is_empty(&self) -> bool { std::ops::Deref::deref(self).is_empty() } - } + use crate::Container; impl Container for Arc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; - - 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<'_> { @@ -398,3 +340,18 @@ pub mod buffer { } } } + +impl WithProgress for Vec { + #[inline(always)] fn count(&self) -> usize { self.len() } + #[inline(always)] fn is_empty(&self) -> bool { Vec::is_empty(self) } +} + +impl WithProgress for std::rc::Rc { + #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } + #[inline(always)] fn is_empty(&self) -> bool { self.as_ref().is_empty() } +} + +impl WithProgress for std::sync::Arc { + #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } + #[inline(always)] fn is_empty(&self) -> bool { self.as_ref().is_empty() } +} diff --git a/timely/examples/columnar.rs b/timely/examples/columnar.rs index 3c03c677f..cb24c0d28 100644 --- a/timely/examples/columnar.rs +++ b/timely/examples/columnar.rs @@ -165,20 +165,11 @@ mod container { } } - impl timely::container::ProgressContainer for Column { + impl timely::container::WithProgress for Column { #[inline(always)] - fn len(&self) -> usize { self.borrow().len() } + fn count(&self) -> usize { self.borrow().len() } } impl timely::Container for Column { - // 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()), - } - } - type ItemRef<'a> = C::Ref<'a>; type Iter<'a> = IterOwn>; fn iter<'a>(&'a self) -> Self::Iter<'a> { self.borrow().into_index_iter() } @@ -287,7 +278,7 @@ mod builder { } } - use timely::container::{ContainerBuilder, LengthPreservingContainerBuilder}; + use timely::container::{ContainerBuilder, CountPreservingContainerBuilder}; impl ContainerBuilder for ColumnBuilder { type Container = Column; @@ -320,5 +311,5 @@ mod builder { } } - impl LengthPreservingContainerBuilder for ColumnBuilder { } + impl CountPreservingContainerBuilder for ColumnBuilder { } } diff --git a/timely/src/dataflow/channels/mod.rs b/timely/src/dataflow/channels/mod.rs index 2f4274508..cbcedaa63 100644 --- a/timely/src/dataflow/channels/mod.rs +++ b/timely/src/dataflow/channels/mod.rs @@ -2,7 +2,7 @@ use serde::{Deserialize, Serialize}; use crate::communication::Push; -use crate::container::ProgressContainer; +use crate::container::WithProgress; /// A collection of types that may be pushed at. pub mod pushers; @@ -32,7 +32,7 @@ impl Message { } } -impl Message { +impl Message { /// Creates a new message instance from arguments. pub fn new(time: T, data: C, from: usize, seq: usize) -> Self { Message { time, data, from, seq } diff --git a/timely/src/dataflow/channels/pact.rs b/timely/src/dataflow/channels/pact.rs index a401c88dd..f93eab5df 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -10,7 +10,7 @@ use std::{fmt::{self, Debug}, marker::PhantomData}; use std::rc::Rc; -use crate::{Container, container::{ContainerBuilder, LengthPreservingContainerBuilder, ProgressContainer, SizableContainer, CapacityContainerBuilder, PushInto}}; +use crate::{Container, container::{ContainerBuilder, CountPreservingContainerBuilder, WithProgress, SizableContainer, CapacityContainerBuilder, PushInto}}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; use crate::communication::{Push, Pull}; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; @@ -34,7 +34,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) { @@ -52,7 +52,7 @@ pub type Exchange = ExchangeCore>, F>; impl ExchangeCore where - CB: LengthPreservingContainerBuilder, + CB: CountPreservingContainerBuilder, for<'a> F: FnMut(&::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. @@ -129,7 +129,7 @@ impl>> LogPusher { } } -impl>> Push> for LogPusher { +impl>> Push> for LogPusher { #[inline] fn push(&mut self, pair: &mut Option>) { if let Some(bundle) = pair { @@ -147,7 +147,7 @@ impl>> Push> for Lo source: self.source, target: self.target, seq_no: self.counter - 1, - length: bundle.data.len(), + length: bundle.data.count(), }) } } @@ -179,7 +179,7 @@ impl>> LogPuller { } } -impl>> Pull> for LogPuller { +impl>> Pull> for LogPuller { #[inline] fn pull(&mut self) -> &mut Option> { let result = self.puller.pull(); @@ -194,7 +194,7 @@ impl>> Pull> for Lo source: bundle.from, target, seq_no: bundle.seq, - length: bundle.data.len(), + length: bundle.data.count(), }); } } diff --git a/timely/src/dataflow/channels/pullers/counter.rs b/timely/src/dataflow/channels/pullers/counter.rs index ec5070630..cac045009 100644 --- a/timely/src/dataflow/channels/pullers/counter.rs +++ b/timely/src/dataflow/channels/pullers/counter.rs @@ -6,7 +6,7 @@ use std::cell::RefCell; use crate::dataflow::channels::Message; use crate::progress::ChangeBatch; use crate::communication::Pull; -use crate::container::ProgressContainer; +use crate::container::WithProgress; /// A wrapper which accounts records pulled past in a shared count map. pub struct Counter>> { @@ -36,7 +36,7 @@ impl Drop for ConsumedGuard { } } -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(), + len: message.data.count(), }; Some((guard, message)) } diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 93e05754a..92520627a 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,7 +2,7 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, ProgressContainer, PushInto}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; diff --git a/timely/src/dataflow/channels/pushers/counter.rs b/timely/src/dataflow/channels/pushers/counter.rs index 1048e8d5c..32c7f9af6 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::ProgressContainer; +use crate::container::WithProgress; /// 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.count() as i64); } // only propagate `None` if dirty (indicates flush) diff --git a/timely/src/dataflow/channels/pushers/tee.rs b/timely/src/dataflow/channels/pushers/tee.rs index ff6ff9daa..2f73f1493 100644 --- a/timely/src/dataflow/channels/pushers/tee.rs +++ b/timely/src/dataflow/channels/pushers/tee.rs @@ -7,7 +7,7 @@ use std::rc::Rc; use crate::dataflow::channels::Message; use crate::communication::Push; use crate::Data; -use crate::container::ProgressContainer; +use crate::container::WithProgress; type PushList = Rc>>>>>; @@ -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(); @@ -39,7 +39,7 @@ impl Push> for Tee { } } -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 2cafaa403..39c69320a 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -1,6 +1,6 @@ //! Operators that separate one stream into two streams based on some condition -use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, Stream, StreamCore}; @@ -94,7 +94,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 42d4bea53..6ec68df30 100644 --- a/timely/src/dataflow/operators/core/capture/capture.rs +++ b/timely/src/dataflow/operators/core/capture/capture.rs @@ -11,14 +11,14 @@ use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; use crate::Data; -use crate::container::ProgressContainer; +use crate::container::WithProgress; 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 @@ -118,7 +118,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/concat.rs b/timely/src/dataflow/operators/core/concat.rs index de3d36c2f..4b36b23f2 100644 --- a/timely/src/dataflow/operators/core/concat.rs +++ b/timely/src/dataflow/operators/core/concat.rs @@ -1,6 +1,6 @@ //! Merges the contents of multiple streams. -use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{StreamCore, Scope}; @@ -23,14 +23,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 +52,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 +62,7 @@ impl Concatenate for StreamCore 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 b0178d0ae..e10345cc6 100644 --- a/timely/src/dataflow/operators/core/enterleave.rs +++ b/timely/src/dataflow/operators/core/enterleave.rs @@ -22,7 +22,7 @@ use std::marker::PhantomData; use std::rc::Rc; -use crate::container::ProgressContainer; +use crate::container::WithProgress; use crate::logging::{TimelyLogger, MessagesEvent}; use crate::progress::Timestamp; use crate::progress::timestamp::Refines; @@ -54,7 +54,7 @@ pub trait Enter, C> { fn enter<'a>(&self, _: &Child<'a, G, T>) -> StreamCore, C>; } -impl, C: Data+ProgressContainer> Enter for StreamCore { +impl, C: Data+WithProgress> Enter for StreamCore { fn enter<'a>(&self, scope: &Child<'a, G, T>) -> StreamCore, C> { use crate::scheduling::Scheduler; @@ -86,7 +86,7 @@ impl, C: Data+ProgressContainer> En } /// Extension trait to move a `Stream` to the parent of its current `Scope`. -pub trait Leave { +pub trait Leave { /// Moves a `Stream` to the parent of its current `Scope`. /// /// # Examples @@ -104,7 +104,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(); @@ -131,14 +131,14 @@ impl> } -struct IngressNub, TContainer: ProgressContainer + Data> { +struct IngressNub, TContainer: WithProgress + Data> { targets: Counter>, phantom: ::std::marker::PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TContainer: ProgressContainer + Data> Push> for IngressNub { +impl, TContainer: WithProgress + Data> 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); @@ -165,7 +165,7 @@ struct EgressNub, TContaine phantom: PhantomData, } -impl Push> for EgressNub +impl Push> for EgressNub where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: Data { fn push(&mut self, message: &mut Option>) { if let Some(inner_message) = message { @@ -208,7 +208,7 @@ impl

LogPusher

{ impl Push> for LogPusher

where - C: ProgressContainer, + C: WithProgress, P: Push>, { fn push(&mut self, element: &mut Option>) { @@ -219,7 +219,7 @@ where source: self.index, target: self.index, seq_no: self.counter, - length: bundle.data.len(), + length: bundle.data.count(), }; let recv_event = MessagesEvent { is_send: false, diff --git a/timely/src/dataflow/operators/core/feedback.rs b/timely/src/dataflow/operators/core/feedback.rs index 779e7b3d1..35847f082 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::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::Data; 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 { +pub struct Handle { builder: OperatorBuilder, summary: ::Summary, output: OutputWrapper, Tee>, diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index a0259c851..78292337e 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,7 +3,7 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, ProgressContainer, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, WithProgress, PushInto}; use crate::scheduling::{Schedule, Activator}; diff --git a/timely/src/dataflow/operators/core/inspect.rs b/timely/src/dataflow/operators/core/inspect.rs index 5a850e29d..18f619f52 100644 --- a/timely/src/dataflow/operators/core/inspect.rs +++ b/timely/src/dataflow/operators/core/inspect.rs @@ -1,6 +1,6 @@ //! Extension trait and implementation for observing and action on streamed data. -use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::{Container, Data}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; @@ -121,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/probe.rs b/timely/src/dataflow/operators/core/probe.rs index 2d00d6d73..a94568a04 100644 --- a/timely/src/dataflow/operators/core/probe.rs +++ b/timely/src/dataflow/operators/core/probe.rs @@ -12,7 +12,7 @@ use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; use crate::dataflow::{StreamCore, Scope}; use crate::Data; -use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; /// Monitors progress at a `Stream`. pub trait Probe { @@ -78,7 +78,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 ac03b61f1..40a03831e 100644 --- a/timely/src/dataflow/operators/core/rc.rs +++ b/timely/src/dataflow/operators/core/rc.rs @@ -2,7 +2,7 @@ use std::rc::Rc; -use crate::container::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::Operator; use crate::dataflow::{Scope, StreamCore}; @@ -26,7 +26,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 e82333800..7122595c0 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::{PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::Data; use crate::order::PartialOrder; use crate::dataflow::{Scope, StreamCore}; @@ -46,11 +46,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/generic/builder_raw.rs b/timely/src/dataflow/operators/generic/builder_raw.rs index 3ecf99131..b84a6214c 100644 --- a/timely/src/dataflow/operators/generic/builder_raw.rs +++ b/timely/src/dataflow/operators/generic/builder_raw.rs @@ -10,7 +10,7 @@ use std::cell::RefCell; use crate::scheduling::{Schedule, Activations}; -use crate::container::ProgressContainer; +use crate::container::WithProgress; use crate::progress::{Source, Target}; use crate::progress::{Timestamp, Operate, operate::SharedProgress, Antichain}; use crate::progress::operate::{Connectivity, PortConnectivity}; @@ -104,7 +104,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller where P: ParallelizationContract { @@ -113,7 +113,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller where P: ParallelizationContract, I: IntoIterator::Summary>)>, @@ -133,14 +133,14 @@ impl OperatorBuilder { } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (Tee, StreamCore) { + pub fn new_output(&mut self) -> (Tee, StreamCore) { let connection = (0 .. self.shape.inputs).map(|i| (i, Antichain::from_elem(Default::default()))); self.new_output_connection(connection) } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) + pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) where I: IntoIterator::Summary>)>, { diff --git a/timely/src/dataflow/operators/generic/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index 4758fbed3..bfe88ca3e 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -9,7 +9,7 @@ use crate::progress::operate::SharedProgress; use crate::progress::frontier::{Antichain, MutableAntichain}; use crate::Container; -use crate::container::{ContainerBuilder, ProgressContainer}; +use crate::container::{ContainerBuilder, WithProgress}; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pushers::Counter as PushCounter; @@ -60,7 +60,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore where P: ParallelizationContract { @@ -76,7 +76,7 @@ impl OperatorBuilder { /// /// Commonly the connections are either the unit summary, indicating the same timestamp might be produced as output, or an empty /// antichain indicating that there is no connection from the input to the output. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore where P: ParallelizationContract, I: IntoIterator::Summary>)> + Clone, diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index eb7e7b581..7682ce9ee 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -16,14 +16,14 @@ 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::{ContainerBuilder, CapacityContainerBuilder, ProgressContainer}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress}; use crate::logging::TimelyLogger as Logger; 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. @@ -38,7 +38,7 @@ pub struct InputHandleCore = InputHandleCore, P>; /// Handle to an operator's input stream and frontier. -pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: ProgressContainer+'a, P: Pull>+'a> { +pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: WithProgress +'a, P: Pull>+'a> { /// The underlying input handle. pub handle: &'a mut InputHandleCore, /// The frontier as reported by timely progress tracking. @@ -48,7 +48,7 @@ pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: ProgressContainer+'a, /// 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. @@ -93,7 +93,7 @@ impl>> InputHandleCore } -impl<'a, T: Timestamp, C: ProgressContainer, P: Pull>+'a> FrontieredInputHandleCore<'a, T, C, P> { +impl<'a, T: Timestamp, C: WithProgress, 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 { @@ -140,13 +140,13 @@ impl<'a, T: Timestamp, C: ProgressContainer, P: Pull>+'a> Frontier } } -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>>, diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index deb92d88c..103074d3e 100644 --- a/timely/src/dataflow/operators/generic/operator.rs +++ b/timely/src/dataflow/operators/generic/operator.rs @@ -13,10 +13,10 @@ use super::builder_rc::OperatorBuilder; use crate::dataflow::operators::generic::OperatorInfo; use crate::dataflow::operators::generic::notificator::{Notificator, FrontierNotificator}; use crate::Data; -use crate::container::{ContainerBuilder, PassthroughContainerBuilder, ProgressContainer}; +use crate::container::{ContainerBuilder, PassthroughContainerBuilder, WithProgress}; /// 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: ProgressContainer + Data, + C2: WithProgress + Data, 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: ProgressContainer + Data, + C2: WithProgress + Data, 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 @@ -389,7 +389,7 @@ impl Operator for StreamCore(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: ProgressContainer + Data, + C2: WithProgress + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, @@ -420,7 +420,7 @@ impl Operator for StreamCore, &mut InputHandleCore, @@ -449,7 +449,7 @@ impl Operator for StreamCore(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: ProgressContainer + Data, + C2: WithProgress + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, @@ -582,7 +582,7 @@ where /// /// }); /// ``` -pub fn empty(scope: &G) -> StreamCore { +pub fn empty(scope: &G) -> StreamCore { let mut builder = OperatorBuilder::new("Empty".to_owned(), scope.clone()); let (_output, stream) = builder.new_output::>(); diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index 0d1e14180..3b744a48b 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -9,7 +9,7 @@ use std::fmt::{self, Debug}; use crate::progress::{Source, Target}; use crate::communication::Push; -use crate::container::ProgressContainer; +use crate::container::WithProgress; use crate::dataflow::Scope; use crate::dataflow::channels::pushers::tee::TeeHelper; use crate::dataflow::channels::Message; @@ -46,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 @@ -75,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]. From 0cdf9de26fa70ff0948058a1431240a607975881 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 24 Jul 2025 12:50:59 +0200 Subject: [PATCH 4/7] Rename Container to IterableContainer and cleanup Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 32 ++++++++----------- timely/examples/columnar.rs | 4 +-- timely/src/dataflow/channels/pact.rs | 14 ++++---- .../src/dataflow/channels/pushers/buffer.rs | 6 ++-- .../src/dataflow/channels/pushers/exchange.rs | 18 +++++------ .../operators/core/capture/extract.rs | 6 ++-- .../dataflow/operators/core/capture/replay.rs | 10 +++--- .../src/dataflow/operators/core/exchange.rs | 7 ++-- timely/src/dataflow/operators/core/filter.rs | 6 ++-- timely/src/dataflow/operators/core/input.rs | 10 +++--- timely/src/dataflow/operators/core/inspect.rs | 8 ++--- timely/src/dataflow/operators/core/map.rs | 12 +++---- timely/src/dataflow/operators/core/ok_err.rs | 6 ++-- .../src/dataflow/operators/core/partition.rs | 6 ++-- .../operators/core/unordered_input.rs | 7 ++-- .../dataflow/operators/generic/builder_rc.rs | 1 - .../src/dataflow/operators/generic/handles.rs | 6 ++-- timely/src/lib.rs | 1 - timely/src/logging.rs | 5 ++- 19 files changed, 78 insertions(+), 87 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 627200c9a..51ce525fe 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -26,19 +26,13 @@ pub trait WithProgress: Default { } /// A container that can reveal its contents through iterating by reference and draining. -pub trait Container: WithProgress { +pub trait IterableContainer: WithProgress { /// 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) - } - /// Iterator type when reading from the container. type Iter<'a>: Iterator> where Self: 'a; @@ -54,7 +48,7 @@ pub trait Container: WithProgress { } /// A container that can be sized and reveals its capacity. -pub trait SizableContainer: Container { +pub trait SizableContainer: WithProgress { /// 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. @@ -110,9 +104,9 @@ pub trait ContainerBuilder: Default + 'static { /// Partitions `container` among `builders`, using the function `index` to direct items. fn partition(container: &mut Self::Container, builders: &mut [Self], mut index: I) where - Self: for<'a> PushInto<::Item<'a>>, - I: for<'a> FnMut(&::Item<'a>) -> usize, - Self::Container: Container, + Self: for<'a> PushInto<::Item<'a>>, + I: for<'a> FnMut(&::Item<'a>) -> usize, + Self::Container: IterableContainer, { for datum in container.drain() { let index = index(&datum); @@ -186,7 +180,7 @@ impl> PushInto for CapacityContainerBuil 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() { @@ -195,7 +189,7 @@ impl> PushInto for CapacityContainerBuil } } -impl ContainerBuilder for CapacityContainerBuilder { +impl ContainerBuilder for CapacityContainerBuilder { type Container = C; #[inline] @@ -218,9 +212,9 @@ impl ContainerBuilder for Capacit } } -impl CountPreservingContainerBuilder for CapacityContainerBuilder { } +impl CountPreservingContainerBuilder for CapacityContainerBuilder { } -impl Container for Vec { +impl IterableContainer for Vec { type ItemRef<'a> = &'a T where T: 'a; type Item<'a> = T where T: 'a; type Iter<'a> = std::slice::Iter<'a, T> where Self: 'a; @@ -278,9 +272,9 @@ mod rc { use std::ops::Deref; use std::rc::Rc; - use crate::Container; + use crate::IterableContainer; - impl Container for Rc { + impl IterableContainer for Rc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; type Iter<'a> = T::Iter<'a> where Self: 'a; @@ -301,9 +295,9 @@ mod arc { use std::ops::Deref; use std::sync::Arc; - use crate::Container; + use crate::IterableContainer; - impl Container for Arc { + impl IterableContainer for Arc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; type Iter<'a> = T::Iter<'a> where Self: 'a; diff --git a/timely/examples/columnar.rs b/timely/examples/columnar.rs index cb24c0d28..3345d8fe9 100644 --- a/timely/examples/columnar.rs +++ b/timely/examples/columnar.rs @@ -2,7 +2,7 @@ use { std::collections::HashMap, - timely::{Container, container::CapacityContainerBuilder}, + timely::container::{CapacityContainerBuilder, IterableContainer}, timely::dataflow::channels::pact::{ExchangeCore, Pipeline}, timely::dataflow::InputHandleCore, timely::dataflow::operators::{Inspect, Operator, Probe}, @@ -169,7 +169,7 @@ mod container { #[inline(always)] fn count(&self) -> usize { self.borrow().len() } } - impl timely::Container for Column { + impl timely::container::IterableContainer 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() } diff --git a/timely/src/dataflow/channels/pact.rs b/timely/src/dataflow/channels/pact.rs index f93eab5df..0e9876650 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -10,7 +10,7 @@ use std::{fmt::{self, Debug}, marker::PhantomData}; use std::rc::Rc; -use crate::{Container, container::{ContainerBuilder, CountPreservingContainerBuilder, WithProgress, SizableContainer, CapacityContainerBuilder, PushInto}}; +use crate::container::{ContainerBuilder, CountPreservingContainerBuilder, IterableContainer, WithProgress, SizableContainer, CapacityContainerBuilder, PushInto}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; use crate::communication::{Push, Pull}; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; @@ -52,8 +52,8 @@ pub type Exchange = ExchangeCore>, F>; impl ExchangeCore where - CB: CountPreservingContainerBuilder, - for<'a> F: FnMut(&::Item<'a>)->u64 + CB: CountPreservingContainerBuilder, + for<'a> F: FnMut(&::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. pub fn new_core(func: F) -> ExchangeCore { @@ -66,7 +66,7 @@ where impl ExchangeCore, F> where - C: SizableContainer, + C: SizableContainer + IterableContainer, for<'a> F: FnMut(&C::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. @@ -81,10 +81,10 @@ 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, - CB: for<'a> PushInto<::Item<'a>>, + CB: ContainerBuilder, + CB: for<'a> PushInto<::Item<'a>>, CB::Container: Data + Send + crate::dataflow::channels::ContainerBytes, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { type Pusher = ExchangePusher>>>, H>; type Puller = LogPuller>>>; diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 92520627a..489ad1062 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,11 +2,11 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto, SizableContainer}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; -use crate::{Container, Data}; +use crate::Data; /// 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/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index 0d145f8d2..8796ff4c4 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -1,17 +1,17 @@ //! The exchange pattern distributes pushed data between many target pushees. use crate::communication::Push; -use crate::container::{ContainerBuilder, PushInto}; +use crate::container::{ContainerBuilder, IterableContainer, PushInto}; use crate::dataflow::channels::Message; -use crate::{Container, Data}; +use crate::Data; // TODO : Software write combining /// Distributes records among target pushees according to a distribution function. pub struct Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { pushers: Vec

, builders: Vec, @@ -21,9 +21,9 @@ where impl Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { /// Allocates a new `Exchange` from a supplied set of pushers and a distribution function. pub fn new(pushers: Vec

, key: H) -> Exchange { @@ -50,10 +50,10 @@ where impl Push> for Exchange where - CB: ContainerBuilder, - CB: for<'a> PushInto<::Item<'a>>, + CB: ContainerBuilder, + CB: for<'a> PushInto<::Item<'a>>, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { #[inline(never)] fn push(&mut self, message: &mut Option>) { diff --git a/timely/src/dataflow/operators/core/capture/extract.rs b/timely/src/dataflow/operators/core/capture/extract.rs index e311dae85..dc89be66e 100644 --- a/timely/src/dataflow/operators/core/capture/extract.rs +++ b/timely/src/dataflow/operators/core/capture/extract.rs @@ -2,7 +2,7 @@ use super::Event; use crate::container::PushInto; -use crate::Container; +use crate::container::IterableContainer; /// Supports extracting a sequence of timestamp and data. pub trait Extract { @@ -49,7 +49,7 @@ pub trait Extract { fn extract(self) -> Vec<(T, C)>; } -impl Extract for ::std::sync::mpsc::Receiver> +impl Extract for ::std::sync::mpsc::Receiver> where for<'a> C: PushInto>, for<'a> C::Item<'a>: Ord, @@ -72,7 +72,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..24df45845 100644 --- a/timely/src/dataflow/operators/core/capture/replay.rs +++ b/timely/src/dataflow/operators/core/capture/replay.rs @@ -38,6 +38,7 @@ //! allowing the replay to occur in a timely dataflow computation with more or fewer workers //! than that in which the stream was captured. +use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Counter as PushCounter; use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; @@ -46,7 +47,6 @@ use crate::progress::Timestamp; use super::Event; use super::event::EventIterator; -use crate::Container; /// Replay a capture stream into a scope with the same timestamp. pub trait Replay : Sized { @@ -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, @@ -76,7 +76,7 @@ where let (targets, stream) = builder.new_output(); - let mut output = PushBuffer::new(PushCounter::new(targets)); + let mut output = PushBuffer::<_,PassthroughContainerBuilder<_>,_>::new(PushCounter::new(targets)); let mut event_streams = self.into_iter().collect::>(); let mut started = false; let mut allocation: C = Default::default(); @@ -100,14 +100,14 @@ where progress.internals[0].extend(vec.into_iter()); }, Owned(Event::Messages(time, mut data)) => { - output.session(&time).give_container(&mut data); + output.session_with_builder(&time).give_container(&mut data); } Borrowed(Event::Progress(vec)) => { progress.internals[0].extend(vec.iter().cloned()); }, Borrowed(Event::Messages(time, data)) => { allocation.clone_from(data); - output.session(time).give_container(&mut allocation); + output.session_with_builder(time).give_container(&mut allocation); } } } diff --git a/timely/src/dataflow/operators/core/exchange.rs b/timely/src/dataflow/operators/core/exchange.rs index bb021a5d5..aea7dee33 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::{IterableContainer, 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,8 @@ pub trait Exchange { impl Exchange for StreamCore where - C: SizableContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, + C: SizableContainer + IterableContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, C: for<'a> PushInto>, - { fn exchange(&self, route: F) -> StreamCore where diff --git a/timely/src/dataflow/operators/core/filter.rs b/timely/src/dataflow/operators/core/filter.rs index 80ef564b0..2dfae083e 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::container::{IterableContainer, SizableContainer, PushInto}; use crate::Data; 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,7 +23,7 @@ pub trait Filter { fn filter)->bool+'static>(&self, predicate: P) -> Self; } -impl Filter for StreamCore +impl Filter for StreamCore where for<'a> C: PushInto> { diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index 78292337e..92dfd957b 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,14 +3,14 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, WithProgress, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, WithProgress, PushInto}; 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::Data; 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 diff --git a/timely/src/dataflow/operators/core/inspect.rs b/timely/src/dataflow/operators/core/inspect.rs index 18f619f52..d088f4619 100644 --- a/timely/src/dataflow/operators/core/inspect.rs +++ b/timely/src/dataflow/operators/core/inspect.rs @@ -1,13 +1,13 @@ //! Extension trait and implementation for observing and action on streamed data. -use crate::container::{PassthroughContainerBuilder, WithProgress}; -use crate::{Container, Data}; +use crate::container::{IterableContainer, PassthroughContainerBuilder, WithProgress}; +use crate::Data; 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 @@ -91,7 +91,7 @@ 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) } diff --git a/timely/src/dataflow/operators/core/map.rs b/timely/src/dataflow/operators/core/map.rs index 8af70e4a4..734db7a62 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::container::{IterableContainer, SizableContainer, PushInto}; use crate::Data; 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: SizableContainer + IterableContainer + PushInto + Data, 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: SizableContainer + IterableContainer + PushInto + Data, 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: SizableContainer + IterableContainer + PushInto + Data, 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..b9b24ba5c 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::container::{IterableContainer, SizableContainer, PushInto}; use crate::Data; 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 @@ -39,7 +39,7 @@ pub trait OkErr { ; } -impl OkErr for StreamCore { +impl OkErr for StreamCore { fn ok_err( &self, mut logic: L, diff --git a/timely/src/dataflow/operators/core/partition.rs b/timely/src/dataflow/operators/core/partition.rs index 754e9182c..9bb9d3d78 100644 --- a/timely/src/dataflow/operators/core/partition.rs +++ b/timely/src/dataflow/operators/core/partition.rs @@ -1,6 +1,6 @@ //! Partition a stream of records into multiple streams. -use timely_container::{Container, ContainerBuilder, PushInto}; +use timely_container::{IterableContainer, ContainerBuilder, PushInto}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; @@ -8,7 +8,7 @@ use crate::dataflow::{Scope, StreamCore}; use crate::Data; /// 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 @@ -34,7 +34,7 @@ pub trait Partition { 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, diff --git a/timely/src/dataflow/operators/core/unordered_input.rs b/timely/src/dataflow/operators/core/unordered_input.rs index 77c6ba81b..169d90146 100644 --- a/timely/src/dataflow/operators/core/unordered_input.rs +++ b/timely/src/dataflow/operators/core/unordered_input.rs @@ -2,8 +2,9 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::{Container, Data}; -use crate::container::{ContainerBuilder, CapacityContainerBuilder}; + +use crate::Data; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, SizableContainer}; 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/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index bfe88ca3e..75f9fa38b 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -8,7 +8,6 @@ use crate::progress::{ChangeBatch, Timestamp}; use crate::progress::operate::SharedProgress; use crate::progress::frontier::{Antichain, MutableAntichain}; -use crate::Container; use crate::container::{ContainerBuilder, WithProgress}; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Tee; diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index 7682ce9ee..1aa7d3f58 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -15,8 +15,8 @@ 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::{ContainerBuilder, CapacityContainerBuilder, WithProgress}; +use crate::Data; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, SizableContainer, WithProgress}; use crate::logging::TimelyLogger as Logger; use crate::dataflow::operators::InputCapability; @@ -235,7 +235,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: SizableContainer + Data, 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/lib.rs b/timely/src/lib.rs index 557b7c7e1..5e115c4ea 100644 --- a/timely/src/lib.rs +++ b/timely/src/lib.rs @@ -65,7 +65,6 @@ pub use timely_communication::Config as CommunicationConfig; pub use worker::Config as WorkerConfig; pub use execute::Config as Config; -pub use timely_container::Container; /// Re-export of the `timely_container` crate. pub mod container { pub use timely_container::*; diff --git a/timely/src/logging.rs b/timely/src/logging.rs index 80777dc98..65980fa90 100644 --- a/timely/src/logging.rs +++ b/timely/src/logging.rs @@ -19,8 +19,7 @@ use std::time::Duration; use columnar::Columnar; use serde::{Deserialize, Serialize}; -use crate::Container; -use crate::container::CapacityContainerBuilder; +use crate::container::{CapacityContainerBuilder, WithProgress}; use crate::dataflow::operators::capture::{Event, EventPusher}; use crate::progress::operate::Connectivity; @@ -31,7 +30,7 @@ pub struct BatchLogger where P: EventPusher { _phantom: ::std::marker::PhantomData, } -impl BatchLogger where P: EventPusher, C: Container { +impl BatchLogger where P: EventPusher, C: WithProgress { /// Creates a new batch logger. pub fn new(event_pusher: P) -> Self { BatchLogger { From 3f78d6b938aa5b0f10e163d8ba0aa451af3aa2fe Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 24 Jul 2025 16:46:52 +0200 Subject: [PATCH 5/7] Remove WithProgress::is_empty Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 15 +++------------ timely/src/dataflow/channels/pushers/buffer.rs | 10 ++++------ .../dataflow/operators/core/capture/extract.rs | 4 +--- timely/src/dataflow/operators/core/filter.rs | 4 +--- timely/src/dataflow/operators/core/input.rs | 10 ++++------ 5 files changed, 13 insertions(+), 30 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 51ce525fe..4230d1168 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -6,7 +6,7 @@ use std::collections::VecDeque; /// A type representing progress, with an update count. /// -/// It describes its update count (`count()`) and whether it is empty (`is_empty()`). +/// It describes its update count (`count()`). /// /// We require [`Default`] for convenience purposes. pub trait WithProgress: Default { @@ -17,12 +17,6 @@ pub trait WithProgress: Default { /// imposed on the [`CountPreservingContainerBuilder`] trait, whose implementors /// must preserve the number of items. fn count(&self) -> usize; - - /// Determine if the container contains any elements, corresponding to `count() == 0`. - #[inline(always)] - fn is_empty(&self) -> bool { - self.count() == 0 - } } /// A container that can reveal its contents through iterating by reference and draining. @@ -90,7 +84,7 @@ 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: WithProgress + Default + Clone + 'static; + type Container: WithProgress + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -204,7 +198,7 @@ impl ContainerBuilder for CapacityContainerBu #[inline] fn finish(&mut self) -> Option<&mut C> { - if !self.current.is_empty() { + if self.current.count() > 0 { self.pending.push_back(std::mem::take(&mut self.current)); } self.empty = self.pending.pop_front(); @@ -337,15 +331,12 @@ pub mod buffer { impl WithProgress for Vec { #[inline(always)] fn count(&self) -> usize { self.len() } - #[inline(always)] fn is_empty(&self) -> bool { Vec::is_empty(self) } } impl WithProgress for std::rc::Rc { #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } - #[inline(always)] fn is_empty(&self) -> bool { self.as_ref().is_empty() } } impl WithProgress for std::sync::Arc { #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } - #[inline(always)] fn is_empty(&self) -> bool { self.as_ref().is_empty() } } diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 489ad1062..ed31f1adf 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,7 +2,7 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto, SizableContainer}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto, SizableContainer}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; @@ -109,11 +109,9 @@ impl>> Buffer)->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 92dfd957b..06c51f970 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,7 +3,7 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, WithProgress, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, PushInto}; use crate::scheduling::{Schedule, Activator}; @@ -439,11 +439,9 @@ impl Handle { /// }); /// ``` pub fn send_batch(&mut self, buffer: &mut CB::Container) { - if !buffer.is_empty() { - // flush buffered elements to ensure local fifo. - self.flush(); - Self::send_container(buffer, &mut self.buffer, &mut self.pushers, &self.now_at); - } + // flush buffered elements to ensure local fifo. + self.flush(); + Self::send_container(buffer, &mut self.buffer, &mut self.pushers, &self.now_at); } /// Advances the current epoch to `next`. From ef54810d1d8d20d23a549d655c003b947c35e6e4 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 24 Jul 2025 16:52:15 +0200 Subject: [PATCH 6/7] Remove WithProgress::is_empty Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 4 ++-- timely/src/dataflow/channels/pushers/buffer.rs | 10 ++++++---- timely/src/dataflow/operators/core/input.rs | 10 ++++++---- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index 4230d1168..bd67cc94c 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -6,7 +6,7 @@ use std::collections::VecDeque; /// A type representing progress, with an update count. /// -/// It describes its update count (`count()`). +/// It describes its update count (`count()`) and whether it is empty (`is_empty()`). /// /// We require [`Default`] for convenience purposes. pub trait WithProgress: Default { @@ -84,7 +84,7 @@ 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: WithProgress + Clone + 'static; + type Container: WithProgress + Default + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index ed31f1adf..5292bc094 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,7 +2,7 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto, SizableContainer}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto, SizableContainer}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; @@ -109,9 +109,11 @@ impl>> Buffer 0 { + self.flush(); + let time = self.time.as_ref().unwrap().clone(); + Message::push_at(container, time, &mut self.pusher); + } } /// An internal implementation of push that should only be called by sessions. diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index 06c51f970..6c15b7b9e 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,7 +3,7 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, WithProgress, PushInto}; use crate::scheduling::{Schedule, Activator}; @@ -439,9 +439,11 @@ impl Handle { /// }); /// ``` pub fn send_batch(&mut self, buffer: &mut CB::Container) { - // flush buffered elements to ensure local fifo. - self.flush(); - Self::send_container(buffer, &mut self.buffer, &mut self.pushers, &self.now_at); + if !buffer.count() > 0 { + // flush buffered elements to ensure local fifo. + self.flush(); + Self::send_container(buffer, &mut self.buffer, &mut self.pushers, &self.now_at); + } } /// Advances the current epoch to `next`. From 236c46bb31466f3409b42c4aced87eeb71928687 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 24 Jul 2025 17:17:28 +0200 Subject: [PATCH 7/7] Restore clear, cleanup Signed-off-by: Moritz Hoffmann --- container/src/lib.rs | 83 ++++++++++++------- timely/examples/columnar.rs | 20 +++-- timely/src/dataflow/channels/mod.rs | 8 +- timely/src/dataflow/channels/pact.rs | 20 ++--- .../src/dataflow/channels/pullers/counter.rs | 4 +- .../src/dataflow/channels/pushers/buffer.rs | 6 +- .../src/dataflow/channels/pushers/counter.rs | 4 +- .../src/dataflow/channels/pushers/exchange.rs | 16 ++-- timely/src/dataflow/channels/pushers/tee.rs | 10 +-- timely/src/dataflow/operators/branch.rs | 6 +- .../operators/core/capture/capture.rs | 7 +- .../operators/core/capture/extract.rs | 5 +- .../dataflow/operators/core/capture/replay.rs | 5 +- timely/src/dataflow/operators/core/concat.rs | 15 ++-- .../src/dataflow/operators/core/enterleave.rs | 19 ++--- .../src/dataflow/operators/core/exchange.rs | 7 +- .../src/dataflow/operators/core/feedback.rs | 18 ++-- timely/src/dataflow/operators/core/filter.rs | 6 +- timely/src/dataflow/operators/core/input.rs | 5 +- timely/src/dataflow/operators/core/inspect.rs | 10 +-- timely/src/dataflow/operators/core/map.rs | 12 +-- timely/src/dataflow/operators/core/ok_err.rs | 6 +- .../src/dataflow/operators/core/partition.rs | 6 +- timely/src/dataflow/operators/core/probe.rs | 10 ++- timely/src/dataflow/operators/core/rc.rs | 11 ++- timely/src/dataflow/operators/core/reclock.rs | 10 +-- .../src/dataflow/operators/core/to_stream.rs | 4 +- .../operators/core/unordered_input.rs | 1 - .../dataflow/operators/generic/builder_raw.rs | 10 +-- .../dataflow/operators/generic/builder_rc.rs | 7 +- .../src/dataflow/operators/generic/handles.rs | 16 ++-- .../dataflow/operators/generic/operator.rs | 26 +++--- timely/src/dataflow/stream.rs | 13 +-- timely/src/lib.rs | 1 + timely/src/logging.rs | 5 +- 35 files changed, 225 insertions(+), 187 deletions(-) diff --git a/container/src/lib.rs b/container/src/lib.rs index bd67cc94c..fa97aff69 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -6,21 +6,25 @@ use std::collections::VecDeque; /// A type representing progress, with an update count. /// -/// It describes its update count (`count()`) and whether it is empty (`is_empty()`). +/// It describes its update count (`count()`). /// /// We require [`Default`] for convenience purposes. -pub trait WithProgress: Default { +pub trait Container: Default { /// The number of elements in this container /// /// 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 [`CountPreservingContainerBuilder`] trait, whose implementors + /// imposed on the [`LengthPreservingContainerBuilder`] trait, whose implementors /// must preserve the number of items. fn count(&self) -> usize; + + /// Remove all contents from `self` while retaining allocated memory. + /// After calling `clear`, `is_empty` must return `true` and `len` 0. + fn clear(&mut self); } /// A container that can reveal its contents through iterating by reference and draining. -pub trait IterableContainer: WithProgress { +pub trait IterContainer: Container { /// The type of elements when reading non-destructively from the container. type ItemRef<'a> where Self: 'a; @@ -42,7 +46,7 @@ pub trait IterableContainer: WithProgress { } /// A container that can be sized and reveals its capacity. -pub trait SizableContainer: WithProgress { +pub trait SizableContainer: Container { /// 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. @@ -84,7 +88,7 @@ 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: WithProgress + Default + Clone + 'static; + type Container: Container + Clone + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -98,14 +102,15 @@ pub trait ContainerBuilder: Default + 'static { /// Partitions `container` among `builders`, using the function `index` to direct items. fn partition(container: &mut Self::Container, builders: &mut [Self], mut index: I) where - Self: for<'a> PushInto<::Item<'a>>, - I: for<'a> FnMut(&::Item<'a>) -> usize, - Self::Container: IterableContainer, + Self: for<'a> PushInto<::Item<'a>>, + I: for<'a> FnMut(&::Item<'a>) -> usize, + Self::Container: IterContainer, { for datum in container.drain() { let index = index(&datum); builders[index].push_into(datum); } + container.clear(); } /// Indicates a good moment to release resources. @@ -122,7 +127,7 @@ pub trait ContainerBuilder: Default + 'static { /// 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. /// If you have any questions about this trait you are best off not implementing it. -pub trait CountPreservingContainerBuilder: ContainerBuilder { } +pub trait LengthPreservingContainerBuilder : ContainerBuilder { } /// A container builder that never produces any outputs, and can be used to pass through data in /// operators. @@ -136,7 +141,7 @@ impl Default for PassthroughContainerBuilder { } } -impl ContainerBuilder for PassthroughContainerBuilder +impl ContainerBuilder for PassthroughContainerBuilder { type Container = C; @@ -183,7 +188,7 @@ impl> PushInto for CapacityContainerBuil } } -impl ContainerBuilder for CapacityContainerBuilder { +impl ContainerBuilder for CapacityContainerBuilder { type Container = C; #[inline] @@ -206,9 +211,14 @@ impl ContainerBuilder for CapacityContainerBu } } -impl CountPreservingContainerBuilder for CapacityContainerBuilder { } +impl LengthPreservingContainerBuilder for CapacityContainerBuilder { } + +impl Container for Vec { + #[inline(always)] fn count(&self) -> usize { Vec::len(self) } + #[inline(always)] fn clear(&mut self) { Vec::clear(self) } +} -impl IterableContainer for Vec { +impl IterContainer for Vec { type ItemRef<'a> = &'a T where T: 'a; type Item<'a> = T where T: 'a; type Iter<'a> = std::slice::Iter<'a, T> where Self: 'a; @@ -266,9 +276,22 @@ mod rc { use std::ops::Deref; use std::rc::Rc; - use crate::IterableContainer; + use crate::{Container, IterContainer}; + + impl Container for Rc { + #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } + #[inline(always)] 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(); + } + } + } + - impl IterableContainer for Rc { + impl IterContainer for Rc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; type Iter<'a> = T::Iter<'a> where Self: 'a; @@ -289,9 +312,21 @@ mod arc { use std::ops::Deref; use std::sync::Arc; - use crate::IterableContainer; + use crate::{Container, IterContainer}; + + impl Container for std::sync::Arc { + #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } + #[inline(always)] 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(); + } + } + } - impl IterableContainer for Arc { + impl IterContainer for Arc { type ItemRef<'a> = T::ItemRef<'a> where Self: 'a; type Item<'a> = T::ItemRef<'a> where Self: 'a; type Iter<'a> = T::Iter<'a> where Self: 'a; @@ -328,15 +363,3 @@ pub mod buffer { } } } - -impl WithProgress for Vec { - #[inline(always)] fn count(&self) -> usize { self.len() } -} - -impl WithProgress for std::rc::Rc { - #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } -} - -impl WithProgress for std::sync::Arc { - #[inline(always)] fn count(&self) -> usize { self.as_ref().count() } -} diff --git a/timely/examples/columnar.rs b/timely/examples/columnar.rs index 3345d8fe9..5026fd9ca 100644 --- a/timely/examples/columnar.rs +++ b/timely/examples/columnar.rs @@ -2,7 +2,7 @@ use { std::collections::HashMap, - timely::container::{CapacityContainerBuilder, IterableContainer}, + timely::container::{CapacityContainerBuilder, IterContainer}, timely::dataflow::channels::pact::{ExchangeCore, Pipeline}, timely::dataflow::InputHandleCore, timely::dataflow::operators::{Inspect, Operator, Probe}, @@ -165,11 +165,19 @@ mod container { } } - impl timely::container::WithProgress for Column { + impl timely::container::Container for Column { + #[inline(always)] fn count(&self) -> usize { self.borrow().len() } + // This sets `self` to be an empty `Typed` variant, appropriate for pushing into. #[inline(always)] - fn count(&self) -> usize { self.borrow().len() } + 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::container::IterableContainer for Column { + 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() } @@ -278,7 +286,7 @@ mod builder { } } - use timely::container::{ContainerBuilder, CountPreservingContainerBuilder}; + use timely::container::{ContainerBuilder, LengthPreservingContainerBuilder}; impl ContainerBuilder for ColumnBuilder { type Container = Column; @@ -311,5 +319,5 @@ mod builder { } } - impl CountPreservingContainerBuilder for ColumnBuilder { } + impl LengthPreservingContainerBuilder for ColumnBuilder { } } diff --git a/timely/src/dataflow/channels/mod.rs b/timely/src/dataflow/channels/mod.rs index cbcedaa63..6a351753b 100644 --- a/timely/src/dataflow/channels/mod.rs +++ b/timely/src/dataflow/channels/mod.rs @@ -2,7 +2,7 @@ use serde::{Deserialize, Serialize}; use crate::communication::Push; -use crate::container::WithProgress; +use crate::Container; /// A collection of types that may be pushed at. pub mod pushers; @@ -32,15 +32,14 @@ impl Message { } } -impl Message { +impl Message { /// Creates a new message instance from arguments. pub fn new(time: T, data: C, from: usize, seq: usize) -> Self { Message { time, data, from, seq } } /// 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's contents are left in an - /// undefined state, specifically the caller cannot rely on this function clearing the buffer. + /// leaves in place, or the container's default element. The buffer is cleared. #[inline] pub fn push_at>>(buffer: &mut C, time: T, pusher: &mut P) { @@ -52,6 +51,7 @@ 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 0e9876650..e7f761048 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -10,7 +10,7 @@ use std::{fmt::{self, Debug}, marker::PhantomData}; use std::rc::Rc; -use crate::container::{ContainerBuilder, CountPreservingContainerBuilder, IterableContainer, WithProgress, SizableContainer, CapacityContainerBuilder, PushInto}; +use crate::container::{ContainerBuilder, LengthPreservingContainerBuilder, IterContainer, Container, SizableContainer, CapacityContainerBuilder, PushInto}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; use crate::communication::{Push, Pull}; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; @@ -34,7 +34,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) { @@ -52,8 +52,8 @@ pub type Exchange = ExchangeCore>, F>; impl ExchangeCore where - CB: CountPreservingContainerBuilder, - for<'a> F: FnMut(&::Item<'a>)->u64 + CB: LengthPreservingContainerBuilder, + for<'a> F: FnMut(&::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. pub fn new_core(func: F) -> ExchangeCore { @@ -66,7 +66,7 @@ where impl ExchangeCore, F> where - C: SizableContainer + IterableContainer, + C: SizableContainer + IterContainer, for<'a> F: FnMut(&C::Item<'a>)->u64 { /// Allocates a new `Exchange` pact from a distribution function. @@ -81,10 +81,10 @@ 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, - CB: for<'a> PushInto<::Item<'a>>, + CB: ContainerBuilder, + CB: for<'a> PushInto<::Item<'a>>, CB::Container: Data + Send + crate::dataflow::channels::ContainerBytes, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { type Pusher = ExchangePusher>>>, H>; type Puller = LogPuller>>>; @@ -129,7 +129,7 @@ impl>> LogPusher { } } -impl>> Push> for LogPusher { +impl>> Push> for LogPusher { #[inline] fn push(&mut self, pair: &mut Option>) { if let Some(bundle) = pair { @@ -179,7 +179,7 @@ impl>> LogPuller { } } -impl>> Pull> for LogPuller { +impl>> Pull> for LogPuller { #[inline] fn pull(&mut self) -> &mut Option> { let result = self.puller.pull(); diff --git a/timely/src/dataflow/channels/pullers/counter.rs b/timely/src/dataflow/channels/pullers/counter.rs index cac045009..4693dfef1 100644 --- a/timely/src/dataflow/channels/pullers/counter.rs +++ b/timely/src/dataflow/channels/pullers/counter.rs @@ -6,7 +6,7 @@ use std::cell::RefCell; use crate::dataflow::channels::Message; use crate::progress::ChangeBatch; use crate::communication::Pull; -use crate::container::WithProgress; +use crate::Container; /// A wrapper which accounts records pulled past in a shared count map. pub struct Counter>> { @@ -36,7 +36,7 @@ impl Drop for ConsumedGuard { } } -impl>> Counter { +impl>> Counter { /// Retrieves the next timestamp and batch of data. #[inline] pub fn next(&mut self) -> Option<&mut Message> { diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 5292bc094..fc6be3055 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -2,11 +2,11 @@ //! with the performance of batched sends. use crate::communication::Push; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, WithProgress, PushInto, SizableContainer}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; -use crate::Data; +use crate::{Container, Data}; /// 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 32c7f9af6..a1f39fbba 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::WithProgress; +use crate::Container; /// A wrapper which updates shared `produced` based on the number of records pushed. #[derive(Debug)] @@ -17,7 +17,7 @@ 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 { diff --git a/timely/src/dataflow/channels/pushers/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index 8796ff4c4..a495f9476 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -1,7 +1,7 @@ //! The exchange pattern distributes pushed data between many target pushees. use crate::communication::Push; -use crate::container::{ContainerBuilder, IterableContainer, PushInto}; +use crate::container::{ContainerBuilder, IterContainer, PushInto}; use crate::dataflow::channels::Message; use crate::Data; @@ -9,9 +9,9 @@ use crate::Data; /// Distributes records among target pushees according to a distribution function. pub struct Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { pushers: Vec

, builders: Vec, @@ -21,9 +21,9 @@ where impl Exchange where - CB: ContainerBuilder, + CB: ContainerBuilder, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { /// Allocates a new `Exchange` from a supplied set of pushers and a distribution function. pub fn new(pushers: Vec

, key: H) -> Exchange { @@ -50,10 +50,10 @@ where impl Push> for Exchange where - CB: ContainerBuilder, - CB: for<'a> PushInto<::Item<'a>>, + CB: ContainerBuilder, + CB: for<'a> PushInto<::Item<'a>>, P: Push>, - for<'a> H: FnMut(&::Item<'a>) -> u64 + for<'a> H: FnMut(&::Item<'a>) -> u64 { #[inline(never)] fn push(&mut self, message: &mut Option>) { diff --git a/timely/src/dataflow/channels/pushers/tee.rs b/timely/src/dataflow/channels/pushers/tee.rs index 2f73f1493..49d9e17d0 100644 --- a/timely/src/dataflow/channels/pushers/tee.rs +++ b/timely/src/dataflow/channels/pushers/tee.rs @@ -5,9 +5,9 @@ use std::fmt::{self, Debug}; use std::rc::Rc; use crate::dataflow::channels::Message; + use crate::communication::Push; -use crate::Data; -use crate::container::WithProgress; +use crate::{Container, Data}; type PushList = Rc>>>>>; @@ -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(); @@ -39,7 +39,7 @@ impl Push> for Tee { } } -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 39c69320a..3798adc83 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -1,10 +1,10 @@ //! Operators that separate one stream into two streams based on some condition -use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, Stream, StreamCore}; -use crate::Data; +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; /// Extension trait for `Stream`. pub trait Branch { @@ -94,7 +94,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 6ec68df30..a0b8b6aae 100644 --- a/timely/src/dataflow/operators/core/capture/capture.rs +++ b/timely/src/dataflow/operators/core/capture/capture.rs @@ -10,15 +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::Data; -use crate::container::WithProgress; +use crate::{Container, Data}; 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 @@ -118,7 +117,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 9d4efba27..49b05d533 100644 --- a/timely/src/dataflow/operators/core/capture/extract.rs +++ b/timely/src/dataflow/operators/core/capture/extract.rs @@ -1,8 +1,7 @@ //! Traits and types for extracting captured timely dataflow streams. use super::Event; -use crate::container::PushInto; -use crate::container::IterableContainer; +use crate::{container::{IterContainer, PushInto}}; /// Supports extracting a sequence of timestamp and data. pub trait Extract { @@ -49,7 +48,7 @@ pub trait Extract { fn extract(self) -> Vec<(T, C)>; } -impl Extract for ::std::sync::mpsc::Receiver> +impl Extract for ::std::sync::mpsc::Receiver> where for<'a> C: PushInto>, for<'a> C::Item<'a>: Ord, diff --git a/timely/src/dataflow/operators/core/capture/replay.rs b/timely/src/dataflow/operators/core/capture/replay.rs index 24df45845..0d6081bf4 100644 --- a/timely/src/dataflow/operators/core/capture/replay.rs +++ b/timely/src/dataflow/operators/core/capture/replay.rs @@ -38,7 +38,7 @@ //! allowing the replay to occur in a timely dataflow computation with more or fewer workers //! than that in which the stream was captured. -use crate::container::{PassthroughContainerBuilder, WithProgress}; +use crate::container::PassthroughContainerBuilder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Counter as PushCounter; use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; @@ -47,6 +47,7 @@ use crate::progress::Timestamp; use super::Event; use super::event::EventIterator; +use crate::Container; /// Replay a capture stream into a scope with the same timestamp. pub trait Replay : Sized { @@ -62,7 +63,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 4b36b23f2..36fc1c71c 100644 --- a/timely/src/dataflow/operators/core/concat.rs +++ b/timely/src/dataflow/operators/core/concat.rs @@ -1,12 +1,13 @@ //! Merges the contents of multiple streams. -use crate::container::{PassthroughContainerBuilder, WithProgress}; -use crate::Data; + +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; 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 +24,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 +53,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 +63,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 e10345cc6..eedca6963 100644 --- a/timely/src/dataflow/operators/core/enterleave.rs +++ b/timely/src/dataflow/operators/core/enterleave.rs @@ -22,12 +22,11 @@ use std::marker::PhantomData; use std::rc::Rc; -use crate::container::WithProgress; use crate::logging::{TimelyLogger, MessagesEvent}; use crate::progress::Timestamp; use crate::progress::timestamp::Refines; use crate::progress::{Source, Target}; -use crate::{Data}; +use crate::{Container, Data}; use crate::communication::Push; use crate::dataflow::channels::pushers::{Counter, Tee}; use crate::dataflow::channels::Message; @@ -36,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> { +pub trait Enter, C: Container> { /// Moves the `Stream` argument into a child of its current `Scope`. /// /// # Examples @@ -54,7 +53,7 @@ pub trait Enter, C> { fn enter<'a>(&self, _: &Child<'a, G, T>) -> StreamCore, C>; } -impl, C: Data+WithProgress> Enter for StreamCore { +impl, C: Data+Container> Enter for StreamCore { fn enter<'a>(&self, scope: &Child<'a, G, T>) -> StreamCore, C> { use crate::scheduling::Scheduler; @@ -86,7 +85,7 @@ impl, C: Data+WithProgress> Enter { +pub trait Leave { /// Moves a `Stream` to the parent of its current `Scope`. /// /// # Examples @@ -104,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(); @@ -131,14 +130,14 @@ impl> Leave } -struct IngressNub, TContainer: WithProgress + Data> { +struct IngressNub, TContainer: Container + Data> { targets: Counter>, phantom: ::std::marker::PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TContainer: WithProgress + Data> Push> for IngressNub { +impl, TContainer: Container + Data> 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); @@ -165,7 +164,7 @@ struct EgressNub, TContaine phantom: PhantomData, } -impl Push> for EgressNub +impl Push> for EgressNub where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: Data { fn push(&mut self, message: &mut Option>) { if let Some(inner_message) = message { @@ -208,7 +207,7 @@ impl

LogPusher

{ impl Push> for LogPusher

where - C: WithProgress, + C: Container, P: Push>, { fn push(&mut self, element: &mut Option>) { diff --git a/timely/src/dataflow/operators/core/exchange.rs b/timely/src/dataflow/operators/core/exchange.rs index aea7dee33..bc04c6d88 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::{IterableContainer, SizableContainer, PushInto}; +use crate::container::{IterContainer, 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,8 +30,9 @@ pub trait Exchange { impl Exchange for StreamCore where - C: SizableContainer + IterableContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, + C: SizableContainer + IterContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, C: 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 35847f082..229dc6939 100644 --- a/timely/src/dataflow/operators/core/feedback.rs +++ b/timely/src/dataflow/operators/core/feedback.rs @@ -1,7 +1,7 @@ //! Create cycles in a timely dataflow graph. -use crate::container::{PassthroughContainerBuilder, WithProgress}; -use crate::Data; +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::operators::generic::OutputWrapper; @@ -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 657a9add4..3e1901d74 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::{IterableContainer, SizableContainer, PushInto}; +use crate::container::{IterContainer, SizableContainer, PushInto}; use crate::Data; 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,7 +23,7 @@ pub trait Filter { fn filter)->bool+'static>(&self, predicate: P) -> Self; } -impl Filter for StreamCore +impl Filter for StreamCore where for<'a> C: PushInto> { diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index 6c15b7b9e..533047b0f 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -3,14 +3,14 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, WithProgress, PushInto}; +use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, PushInto}; use crate::scheduling::{Schedule, Activator}; use crate::progress::{Operate, operate::SharedProgress, Timestamp, ChangeBatch}; use crate::progress::Source; use crate::progress::operate::Connectivity; -use crate::Data; +use crate::{Container, Data}; use crate::communication::Push; use crate::dataflow::{Scope, ScopeParent, StreamCore}; use crate::dataflow::channels::pushers::{Tee, Counter}; @@ -392,6 +392,7 @@ 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 d088f4619..305e700ff 100644 --- a/timely/src/dataflow/operators/core/inspect.rs +++ b/timely/src/dataflow/operators/core/inspect.rs @@ -1,13 +1,13 @@ //! Extension trait and implementation for observing and action on streamed data. -use crate::container::{IterableContainer, PassthroughContainerBuilder, WithProgress}; -use crate::Data; +use crate::{Container, Data}; +use crate::container::{IterContainer, PassthroughContainerBuilder}; 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 @@ -91,7 +91,7 @@ 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) } @@ -121,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 734db7a62..adfd1063c 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::{IterableContainer, SizableContainer, PushInto}; +use crate::container::{IterContainer, SizableContainer, PushInto}; use crate::Data; 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 + IterableContainer + PushInto + Data, + C2: SizableContainer + IterContainer + PushInto + Data, 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 + IterableContainer + PushInto + Data, + C2: SizableContainer + IterContainer + PushInto + Data, 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 + IterableContainer + PushInto + Data, + C2: SizableContainer + IterContainer + PushInto + Data, 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 b9b24ba5c..e8d1aa835 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::{IterableContainer, SizableContainer, PushInto}; +use crate::container::{IterContainer, SizableContainer, PushInto}; use crate::Data; 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 @@ -39,7 +39,7 @@ pub trait OkErr { ; } -impl OkErr for StreamCore { +impl OkErr for StreamCore { fn ok_err( &self, mut logic: L, diff --git a/timely/src/dataflow/operators/core/partition.rs b/timely/src/dataflow/operators/core/partition.rs index 9bb9d3d78..6517ab94a 100644 --- a/timely/src/dataflow/operators/core/partition.rs +++ b/timely/src/dataflow/operators/core/partition.rs @@ -1,6 +1,6 @@ //! Partition a stream of records into multiple streams. -use timely_container::{IterableContainer, ContainerBuilder, PushInto}; +use timely_container::{IterContainer, ContainerBuilder, PushInto}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; @@ -8,7 +8,7 @@ use crate::dataflow::{Scope, StreamCore}; use crate::Data; /// 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 @@ -34,7 +34,7 @@ pub trait Partition { 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, diff --git a/timely/src/dataflow/operators/core/probe.rs b/timely/src/dataflow/operators/core/probe.rs index a94568a04..997b3922e 100644 --- a/timely/src/dataflow/operators/core/probe.rs +++ b/timely/src/dataflow/operators/core/probe.rs @@ -10,12 +10,14 @@ use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; + + use crate::dataflow::{StreamCore, Scope}; -use crate::Data; -use crate::container::{PassthroughContainerBuilder, WithProgress}; +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; /// Monitors progress at a `Stream`. -pub trait Probe { +pub trait Probe { /// Constructs a progress probe which indicates which timestamps have elapsed at the operator. /// /// # Examples @@ -78,7 +80,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 40a03831e..392637d22 100644 --- a/timely/src/dataflow/operators/core/rc.rs +++ b/timely/src/dataflow/operators/core/rc.rs @@ -1,15 +1,14 @@ //! Shared containers -use std::rc::Rc; - -use crate::container::{PassthroughContainerBuilder, WithProgress}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::Operator; use crate::dataflow::{Scope, StreamCore}; -use crate::Data; +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; +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 @@ -26,7 +25,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 7122595c0..d592669f6 100644 --- a/timely/src/dataflow/operators/core/reclock.rs +++ b/timely/src/dataflow/operators/core/reclock.rs @@ -1,7 +1,7 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::container::{PassthroughContainerBuilder, WithProgress}; -use crate::Data; +use crate::{Container, Data}; +use crate::container::PassthroughContainerBuilder; use crate::order::PartialOrder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; @@ -46,11 +46,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 3b340ba65..cfc6f429b 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::Data; +use crate::{Container, Data}; 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 diff --git a/timely/src/dataflow/operators/core/unordered_input.rs b/timely/src/dataflow/operators/core/unordered_input.rs index 169d90146..72d9c6ad4 100644 --- a/timely/src/dataflow/operators/core/unordered_input.rs +++ b/timely/src/dataflow/operators/core/unordered_input.rs @@ -2,7 +2,6 @@ use std::rc::Rc; use std::cell::RefCell; - use crate::Data; use crate::container::{ContainerBuilder, CapacityContainerBuilder, SizableContainer}; diff --git a/timely/src/dataflow/operators/generic/builder_raw.rs b/timely/src/dataflow/operators/generic/builder_raw.rs index b84a6214c..4659383a9 100644 --- a/timely/src/dataflow/operators/generic/builder_raw.rs +++ b/timely/src/dataflow/operators/generic/builder_raw.rs @@ -10,10 +10,10 @@ use std::cell::RefCell; use crate::scheduling::{Schedule, Activations}; -use crate::container::WithProgress; use crate::progress::{Source, Target}; use crate::progress::{Timestamp, Operate, operate::SharedProgress, Antichain}; use crate::progress::operate::{Connectivity, PortConnectivity}; +use crate::Container; use crate::dataflow::{StreamCore, Scope}; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pact::ParallelizationContract; @@ -104,7 +104,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller where P: ParallelizationContract { @@ -113,7 +113,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> P::Puller where P: ParallelizationContract, I: IntoIterator::Summary>)>, @@ -133,14 +133,14 @@ impl OperatorBuilder { } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (Tee, StreamCore) { + pub fn new_output(&mut self) -> (Tee, StreamCore) { let connection = (0 .. self.shape.inputs).map(|i| (i, Antichain::from_elem(Default::default()))); self.new_output_connection(connection) } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) + pub fn new_output_connection(&mut self, connection: I) -> (Tee, StreamCore) where I: IntoIterator::Summary>)>, { diff --git a/timely/src/dataflow/operators/generic/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index 75f9fa38b..66957ddf5 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -8,7 +8,8 @@ use crate::progress::{ChangeBatch, Timestamp}; use crate::progress::operate::SharedProgress; use crate::progress::frontier::{Antichain, MutableAntichain}; -use crate::container::{ContainerBuilder, WithProgress}; +use crate::Container; +use crate::container::ContainerBuilder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pushers::Counter as PushCounter; @@ -59,7 +60,7 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore where P: ParallelizationContract { @@ -75,7 +76,7 @@ impl OperatorBuilder { /// /// Commonly the connections are either the unit summary, indicating the same timestamp might be produced as output, or an empty /// antichain indicating that there is no connection from the input to the output. - pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: I) -> InputHandleCore where P: ParallelizationContract, I: IntoIterator::Summary>)> + Clone, diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index 1aa7d3f58..09cd20733 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -15,15 +15,15 @@ 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::Data; -use crate::container::{ContainerBuilder, CapacityContainerBuilder, SizableContainer, WithProgress}; +use crate::{Container, Data}; +use crate::container::{ContainerBuilder, CapacityContainerBuilder, SizableContainer}; use crate::logging::TimelyLogger as Logger; 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. @@ -38,7 +38,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: WithProgress +'a, P: Pull>+'a> { +pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: Container+'a, P: Pull>+'a> { /// The underlying input handle. pub handle: &'a mut InputHandleCore, /// The frontier as reported by timely progress tracking. @@ -48,7 +48,7 @@ pub struct FrontieredInputHandleCore<'a, T: Timestamp, C: WithProgress +'a, P: P /// 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. @@ -93,7 +93,7 @@ impl>> InputHandleCore>+'a> FrontieredInputHandleCore<'a, T, C, P> { +impl<'a, T: Timestamp, C: Container, 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 { @@ -140,13 +140,13 @@ impl<'a, T: Timestamp, C: WithProgress, P: Pull>+'a> FrontieredInp } } -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>>, diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index 103074d3e..4b17a891c 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::Data; -use crate::container::{ContainerBuilder, PassthroughContainerBuilder, WithProgress}; +use crate::{Container, Data}; +use crate::container::{ContainerBuilder, PassthroughContainerBuilder}; /// 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: WithProgress + Data, + C2: Container + Data, 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: WithProgress + Data, + C2: Container + Data, 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 @@ -389,7 +389,7 @@ impl Operator for StreamCore { fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: WithProgress + Data, + C2: Container + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, @@ -420,7 +420,7 @@ impl Operator for StreamCore { stream } - fn binary_notify, &mut InputHandleCore, @@ -449,7 +449,7 @@ impl Operator for StreamCore { fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: WithProgress + Data, + C2: Container + Data, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, @@ -582,12 +582,14 @@ where /// /// }); /// ``` -pub fn empty(scope: &G) -> StreamCore { +pub fn empty(scope: &G) -> StreamCore { let mut builder = OperatorBuilder::new("Empty".to_owned(), scope.clone()); let (_output, stream) = builder.new_output::>(); builder.set_notify(false); - builder.build(|_caps| |_frontier| { }); + builder.build(|_caps| |_frontier| { + // drop capability, do nothing + }); stream } diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index 3b744a48b..7959c842a 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -4,17 +4,18 @@ //! operator output. Extension methods on the `Stream` type provide the appearance of higher-level //! declarative programming, while constructing a dataflow graph underneath. -use std::fmt::{self, Debug}; - use crate::progress::{Source, Target}; use crate::communication::Push; -use crate::container::WithProgress; 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; -/// Abstraction of a stream of `C: ProgressContainer` records timestamped with `S::Timestamp`. +/// Abstraction of a stream of `C: Container` records timestamped with `S::Timestamp`. /// /// Internally `Stream` maintains a list of data recipients who should be presented with data /// produced by the source of the stream. @@ -46,7 +47,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 @@ -75,7 +76,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 5e115c4ea..557b7c7e1 100644 --- a/timely/src/lib.rs +++ b/timely/src/lib.rs @@ -65,6 +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; /// Re-export of the `timely_container` crate. pub mod container { pub use timely_container::*; diff --git a/timely/src/logging.rs b/timely/src/logging.rs index 65980fa90..80777dc98 100644 --- a/timely/src/logging.rs +++ b/timely/src/logging.rs @@ -19,7 +19,8 @@ use std::time::Duration; use columnar::Columnar; use serde::{Deserialize, Serialize}; -use crate::container::{CapacityContainerBuilder, WithProgress}; +use crate::Container; +use crate::container::CapacityContainerBuilder; use crate::dataflow::operators::capture::{Event, EventPusher}; use crate::progress::operate::Connectivity; @@ -30,7 +31,7 @@ pub struct BatchLogger where P: EventPusher { _phantom: ::std::marker::PhantomData, } -impl BatchLogger where P: EventPusher, C: WithProgress { +impl BatchLogger where P: EventPusher, C: Container { /// Creates a new batch logger. pub fn new(event_pusher: P) -> Self { BatchLogger {