diff --git a/container/src/lib.rs b/container/src/lib.rs index d48e45015..3e8293b22 100644 --- a/container/src/lib.rs +++ b/container/src/lib.rs @@ -14,10 +14,6 @@ pub mod flatcontainer; /// /// 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; @@ -105,7 +101,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: Container + 'static; /// Extract assembled containers, potentially leaving unfinished data behind. Can /// be called repeatedly, for example while the caller can send data. /// @@ -170,7 +166,7 @@ impl> PushInto for CapacityContainerBuil } } -impl ContainerBuilder for CapacityContainerBuilder { +impl ContainerBuilder for CapacityContainerBuilder { type Container = C; #[inline] diff --git a/mdbook/src/chapter_2/chapter_2_3.md b/mdbook/src/chapter_2/chapter_2_3.md index 71e5774be..46b4d2b16 100644 --- a/mdbook/src/chapter_2/chapter_2_3.md +++ b/mdbook/src/chapter_2/chapter_2_3.md @@ -126,12 +126,12 @@ use timely::dataflow::operators::{ToStream, Partition, Inspect}; fn main() { timely::example(|scope| { - let streams = (0..10).to_stream(scope) + let mut streams = (0..10).to_stream(scope) .partition(3, |x| (x % 3, x)); - streams[0].inspect(|x| println!("seen 0: {:?}", x)); - streams[1].inspect(|x| println!("seen 1: {:?}", x)); - streams[2].inspect(|x| println!("seen 2: {:?}", x)); + streams.pop().unwrap().inspect(|x| println!("seen 2: {:?}", x)); + streams.pop().unwrap().inspect(|x| println!("seen 1: {:?}", x)); + streams.pop().unwrap().inspect(|x| println!("seen 0: {:?}", x)); }); } ``` @@ -147,11 +147,11 @@ use timely::dataflow::operators::{ToStream, Partition, Concat, Inspect}; fn main() { timely::example(|scope| { - let streams = (0..10).to_stream(scope) + let mut streams = (0..10).to_stream(scope) .partition(3, |x| (x % 3, x)); - streams[0] - .concat(&streams[1]) - .concat(&streams[2]) + streams.pop().unwrap() + .concat(streams.pop().unwrap()) + .concat(streams.pop().unwrap()) .inspect(|x| println!("seen: {:?}", x)); }); } @@ -169,7 +169,8 @@ fn main() { let streams = (0..10).to_stream(scope) .partition(3, |x| (x % 3, x)); - scope.concatenate(streams) + scope.clone() + .concatenate(streams) .inspect(|x| println!("seen: {:?}", x)); }); } diff --git a/mdbook/src/chapter_2/chapter_2_4.md b/mdbook/src/chapter_2/chapter_2_4.md index dbbf7fbc1..19c76326b 100644 --- a/mdbook/src/chapter_2/chapter_2_4.md +++ b/mdbook/src/chapter_2/chapter_2_4.md @@ -182,7 +182,7 @@ fn main() { let in1 = (0 .. 10).to_stream(scope); let in2 = (0 .. 10).to_stream(scope); - in1.binary_frontier(&in2, Pipeline, Pipeline, "concat_buffer", |capability, info| { + in1.binary_frontier(in2, Pipeline, Pipeline, "concat_buffer", |capability, info| { let mut notificator = FrontierNotificator::default(); let mut stash = HashMap::new(); @@ -233,7 +233,7 @@ fn main() { let in1 = (0 .. 10).to_stream(scope); let in2 = (0 .. 10).to_stream(scope); - in1.binary_frontier(&in2, Pipeline, Pipeline, "concat_buffer", |capability, info| { + in1.binary_frontier(in2, Pipeline, Pipeline, "concat_buffer", |capability, info| { let mut stash = HashMap::new(); diff --git a/mdbook/src/chapter_4/chapter_4_2.md b/mdbook/src/chapter_4/chapter_4_2.md index 7c057da38..45230367a 100644 --- a/mdbook/src/chapter_4/chapter_4_2.md +++ b/mdbook/src/chapter_4/chapter_4_2.md @@ -24,7 +24,7 @@ fn main() { // circulate numbers, Collatz stepping each time. (1 .. 10) .to_stream(scope) - .concat(&stream) + .concat(stream) .map(|x| if x % 2 == 0 { x / 2 } else { 3 * x + 1 } ) .inspect(|x| println!("{:?}", x)) .filter(|x| *x != 1) @@ -63,17 +63,17 @@ fn main() { let results1 = stream1.map(|x| 3 * x + 1); // partition the input and feedback streams by even-ness. - let parts = + let mut parts = (1 .. 10) .to_stream(scope) - .concat(&results0) - .concat(&results1) + .concat(results0) + .concat(results1) .inspect(|x| println!("{:?}", x)) .partition(2, |x| (x % 2, x)); // connect each part appropriately. - parts[0].connect_loop(handle0); - parts[1].connect_loop(handle1); + parts.pop().unwrap().connect_loop(handle1); + parts.pop().unwrap().connect_loop(handle0); }); } ``` @@ -103,7 +103,7 @@ fn main() { input .enter(subscope) - .concat(&stream) + .concat(stream) .map(|x| if x % 2 == 0 { x / 2 } else { 3 * x + 1 } ) .inspect(|x| println!("{:?}", x)) .filter(|x| *x != 1) diff --git a/mdbook/src/chapter_4/chapter_4_3.md b/mdbook/src/chapter_4/chapter_4_3.md index beeabd834..32a0e4a56 100644 --- a/mdbook/src/chapter_4/chapter_4_3.md +++ b/mdbook/src/chapter_4/chapter_4_3.md @@ -76,7 +76,7 @@ fn main() { // Assign timestamps to records so that not much work is in each time. .delay(|number, time| number / 100 ) // Buffer records until all prior timestamps have completed. - .binary_frontier(&cycle, Pipeline, Pipeline, "Buffer", move |capability, info| { + .binary_frontier(cycle, Pipeline, Pipeline, "Buffer", move |capability, info| { move |input1, input2, output| { diff --git a/timely/examples/bfs.rs b/timely/examples/bfs.rs index 1f29f3962..c8a8cc351 100644 --- a/timely/examples/bfs.rs +++ b/timely/examples/bfs.rs @@ -45,7 +45,7 @@ fn main() { // use the stream of edges graph.binary_notify( - &stream, + stream, Exchange::new(|x: &(u32, u32)| u64::from(x.0)), Exchange::new(|x: &(u32, u32)| u64::from(x.0)), "BFS", @@ -130,7 +130,7 @@ fn main() { }); } ) - .concat(&(0..1).map(|x| (x,x)).to_stream(scope)) + .concat((0..1).map(|x| (x,x)).to_stream(scope)) .connect_loop(handle); }); }).unwrap(); // asserts error-free execution; diff --git a/timely/examples/hashjoin.rs b/timely/examples/hashjoin.rs index c5383f993..706bdc422 100644 --- a/timely/examples/hashjoin.rs +++ b/timely/examples/hashjoin.rs @@ -32,7 +32,7 @@ fn main() { let exchange2 = Exchange::new(|x: &(u64, u64)| x.0); stream1 - .binary(&stream2, exchange1, exchange2, "HashJoin", |_capability, _info| { + .binary(stream2, exchange1, exchange2, "HashJoin", |_capability, _info| { let mut map1 = HashMap::>::new(); let mut map2 = HashMap::>::new(); diff --git a/timely/examples/loopdemo.rs b/timely/examples/loopdemo.rs index df7dc3fe8..5681a21c9 100644 --- a/timely/examples/loopdemo.rs +++ b/timely/examples/loopdemo.rs @@ -27,12 +27,12 @@ fn main() { let step = stream - .concat(&loop_stream) + .concat(loop_stream) .map(|x| if x % 2 == 0 { x / 2 } else { 3 * x + 1 }) .filter(|x| x > &1); - - step.connect_loop(loop_handle); - step.probe_with(&mut probe); + step + .probe_with(&mut probe) + .connect_loop(loop_handle); }); let ns_per_request = 1_000_000_000 / rate; diff --git a/timely/examples/pagerank.rs b/timely/examples/pagerank.rs index 8c9cdb08f..301719f79 100644 --- a/timely/examples/pagerank.rs +++ b/timely/examples/pagerank.rs @@ -23,7 +23,7 @@ fn main() { // bring edges and ranks together! let changes = edge_stream.binary_frontier( - &rank_stream, + rank_stream, Exchange::new(|x: &((usize, usize), i64)| (x.0).0 as u64), Exchange::new(|x: &(usize, i64)| x.0 as u64), "PageRank", diff --git a/timely/examples/pingpong.rs b/timely/examples/pingpong.rs index 2104de289..edd644d38 100644 --- a/timely/examples/pingpong.rs +++ b/timely/examples/pingpong.rs @@ -14,7 +14,7 @@ fn main() { (0 .. elements) .filter(move |&x| (x as usize) % peers == index) .to_stream(scope) - .concat(&cycle) + .concat(cycle) .exchange(|&x| x) .map_in_place(|x| *x += 1) .branch_when(move |t| t < &iterations).1 diff --git a/timely/examples/unionfind.rs b/timely/examples/unionfind.rs index 04641c174..5e827049b 100644 --- a/timely/examples/unionfind.rs +++ b/timely/examples/unionfind.rs @@ -48,11 +48,11 @@ fn main() { } trait UnionFind { - fn union_find(&self) -> Self; + fn union_find(self) -> Self; } impl UnionFind for Stream { - fn union_find(&self) -> Stream { + fn union_find(self) -> Stream { self.unary(Pipeline, "UnionFind", |_,_| { diff --git a/timely/src/dataflow/channels/pact.rs b/timely/src/dataflow/channels/pact.rs index e93cb3b68..fe04f8cd1 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -18,7 +18,6 @@ use crate::dataflow::channels::Message; use crate::logging::{TimelyLogger as Logger, MessagesEvent}; use crate::progress::Timestamp; use crate::worker::AsWorker; -use crate::Data; /// A `ParallelizationContract` allocates paired `Push` and `Pull` implementors. pub trait ParallelizationContract { @@ -84,7 +83,7 @@ impl ParallelizationContract for where CB: ContainerBuilder, CB: for<'a> PushInto<::Item<'a>>, - CB::Container: Data + Send + SizableContainer + crate::dataflow::channels::ContainerBytes, + CB::Container: Send + SizableContainer + crate::dataflow::channels::ContainerBytes, for<'a> H: FnMut(&::Item<'a>) -> u64 { type Pusher = ExchangePusher>>>, H>; diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 4ea872fb7..e34c21978 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -6,7 +6,7 @@ use crate::container::{ContainerBuilder, CapacityContainerBuilder, PushInto}; use crate::dataflow::channels::Message; use crate::dataflow::operators::Capability; use crate::progress::Timestamp; -use crate::{Container, Data}; +use crate::Container; /// 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, P> { @@ -133,7 +133,7 @@ pub struct Session<'a, T, CB, P> { buffer: &'a mut Buffer, } -impl<'a, T, C: Container + Data, P> Session<'a, T, CapacityContainerBuilder, P> +impl<'a, T, C: Container + 'static, P> Session<'a, T, CapacityContainerBuilder, P> where T: Eq + Clone + 'a, P: Push> + 'a, diff --git a/timely/src/dataflow/channels/pushers/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index a38f87aa5..89b1ddfa0 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -3,7 +3,7 @@ use crate::communication::Push; use crate::container::{ContainerBuilder, SizableContainer, PushInto}; use crate::dataflow::channels::Message; -use crate::{Container, Data}; +use crate::Container; // TODO : Software write combining /// Distributes records among target pushees according to a distribution function. @@ -50,7 +50,7 @@ where } } -impl Push> for Exchange +impl Push> for Exchange where CB: ContainerBuilder, CB::Container: SizableContainer, diff --git a/timely/src/dataflow/channels/pushers/mod.rs b/timely/src/dataflow/channels/pushers/mod.rs index 295d033ca..fa6ee3b25 100644 --- a/timely/src/dataflow/channels/pushers/mod.rs +++ b/timely/src/dataflow/channels/pushers/mod.rs @@ -1,7 +1,9 @@ -pub use self::tee::{Tee, TeeHelper}; -pub use self::exchange::Exchange; pub use self::counter::Counter; +pub use self::exchange::Exchange; +pub use self::owned::PushOwned; +pub use self::tee::{Tee, TeeHelper}; +pub mod owned; pub mod tee; pub mod exchange; pub mod counter; diff --git a/timely/src/dataflow/channels/pushers/owned.rs b/timely/src/dataflow/channels/pushers/owned.rs new file mode 100644 index 000000000..677535724 --- /dev/null +++ b/timely/src/dataflow/channels/pushers/owned.rs @@ -0,0 +1,43 @@ +//! A `Push` implementor with a single target. + +use std::cell::RefCell; +use std::fmt; +use std::rc::Rc; + +use timely_communication::Push; +use crate::dataflow::channels::Message; + +/// A pusher that can bind to a single downstream pusher. +pub struct PushOwned(Rc>>>>>); + +impl PushOwned { + /// Create a new `PushOwned`. Similarly to `Tee`, it returns a pair where either element + /// can be used as pusher or registrar. + pub fn new() -> (Self, Self) { + let shared = Rc::new(RefCell::new(None)); + (Self(Rc::clone(&shared)), Self(shared)) + } + + /// Set the downstream pusher. + /// + /// Consumes `Self` as only a single pusher can be set. + pub fn set_pusher> + 'static>(self, pusher: P) { + *self.0.borrow_mut() = Some(Box::new(pusher)); + } +} + +impl fmt::Debug for PushOwned { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("PushOwned").finish_non_exhaustive() + } +} + +impl Push> for PushOwned { + #[inline] + fn push(&mut self, message: &mut Option>) { + let mut pusher = self.0.borrow_mut(); + if let Some(pusher) = pusher.as_mut() { + pusher.push(message); + } + } +} diff --git a/timely/src/dataflow/channels/pushers/tee.rs b/timely/src/dataflow/channels/pushers/tee.rs index e9251c24a..7b0ffa817 100644 --- a/timely/src/dataflow/channels/pushers/tee.rs +++ b/timely/src/dataflow/channels/pushers/tee.rs @@ -7,9 +7,10 @@ use std::rc::Rc; use crate::dataflow::channels::Message; use crate::communication::Push; -use crate::{Container, Data}; +use crate::Container; +use crate::dataflow::channels::pushers::PushOwned; -type PushList = Rc>>>>>; +type PushList = Rc>>>; /// Wraps a shared list of `Box` to forward pushes to. Owned by `Stream`. pub struct Tee { @@ -17,7 +18,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(); @@ -86,8 +87,8 @@ pub struct TeeHelper { impl TeeHelper { /// Adds a new `Push` implementor to the list of recipients shared with a `Stream`. - pub fn add_pusher>+'static>(&self, pusher: P) { - self.shared.borrow_mut().push(Box::new(pusher)); + pub fn add_pusher(&self, pusher: PushOwned) { + self.shared.borrow_mut().push(pusher); } } diff --git a/timely/src/dataflow/mod.rs b/timely/src/dataflow/mod.rs index 043317fd8..4ec963eb1 100644 --- a/timely/src/dataflow/mod.rs +++ b/timely/src/dataflow/mod.rs @@ -13,7 +13,7 @@ //! }); //! ``` -pub use self::stream::{StreamCore, Stream}; +pub use self::stream::{StreamTee, Stream, StreamCore}; pub use self::scopes::{Scope, ScopeParent}; pub use self::operators::core::input::Handle as InputHandleCore; diff --git a/timely/src/dataflow/operators/aggregation/aggregate.rs b/timely/src/dataflow/operators/aggregation/aggregate.rs index 196f1c5d0..63ad6b07a 100644 --- a/timely/src/dataflow/operators/aggregation/aggregate.rs +++ b/timely/src/dataflow/operators/aggregation/aggregate.rs @@ -2,7 +2,7 @@ use std::hash::Hash; use std::collections::HashMap; -use crate::{Data, ExchangeData}; +use crate::ExchangeData; use crate::dataflow::{Stream, Scope}; use crate::dataflow::operators::generic::operator::Operator; use crate::dataflow::channels::pact::Exchange; @@ -60,17 +60,17 @@ pub trait Aggregate { /// .inspect(|x| assert!(*x == (0, 5) || *x == (1, 5))); /// }); /// ``` - fn aggregateR+'static, H: Fn(&K)->u64+'static>( - &self, + fn aggregateR+'static, H: Fn(&K)->u64+'static>( + self, fold: F, emit: E, hash: H) -> Stream where S::Timestamp: Eq; } -impl Aggregate for Stream { +impl Aggregate for Stream { - fn aggregateR+'static, H: Fn(&K)->u64+'static>( - &self, + fn aggregateR+'static, H: Fn(&K)->u64+'static>( + self, fold: F, emit: E, hash: H) -> Stream where S::Timestamp: Eq { diff --git a/timely/src/dataflow/operators/aggregation/state_machine.rs b/timely/src/dataflow/operators/aggregation/state_machine.rs index 7347f8c12..b75ab174b 100644 --- a/timely/src/dataflow/operators/aggregation/state_machine.rs +++ b/timely/src/dataflow/operators/aggregation/state_machine.rs @@ -2,7 +2,7 @@ use std::hash::Hash; use std::collections::HashMap; -use crate::{Data, ExchangeData}; +use crate::ExchangeData; use crate::dataflow::{Stream, Scope}; use crate::dataflow::operators::generic::operator::Operator; use crate::dataflow::channels::pact::Exchange; @@ -46,22 +46,22 @@ pub trait StateMachine { /// }); /// ``` fn state_machine< - R: Data, // output type + R: 'static, // output type D: Default+'static, // per-key state (data) I: IntoIterator, // type of output iterator F: Fn(&K, V, &mut D)->(bool, I)+'static, // state update logic H: Fn(&K)->u64+'static, // "hash" function for keys - >(&self, fold: F, hash: H) -> Stream where S::Timestamp : Hash+Eq ; + >(self, fold: F, hash: H) -> Stream where S::Timestamp : Hash+Eq ; } -impl StateMachine for Stream { +impl StateMachine for Stream { fn state_machine< - R: Data, // output type + R: 'static, // output type D: Default+'static, // per-key state (data) I: IntoIterator, // type of output iterator F: Fn(&K, V, &mut D)->(bool, I)+'static, // state update logic H: Fn(&K)->u64+'static, // "hash" function for keys - >(&self, fold: F, hash: H) -> Stream where S::Timestamp : Hash+Eq { + >(self, fold: F, hash: H) -> Stream where S::Timestamp : Hash+Eq { let mut pending: HashMap<_, Vec<(K, V)>> = HashMap::new(); // times -> (keys -> state) let mut states = HashMap::new(); // keys -> state diff --git a/timely/src/dataflow/operators/branch.rs b/timely/src/dataflow/operators/branch.rs index c3865cb2b..05a1396fd 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -3,10 +3,10 @@ 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::Container; /// Extension trait for `Stream`. -pub trait Branch { +pub trait Branch { /// Takes one input stream and splits it into two output streams. /// For each record, the supplied closure is called with a reference to /// the data and its time. If it returns `true`, the record will be sent @@ -29,14 +29,14 @@ pub trait Branch { /// }); /// ``` fn branch( - &self, + self, condition: impl Fn(&S::Timestamp, &D) -> bool + 'static, ) -> (Stream, Stream); } -impl Branch for Stream { +impl Branch for Stream { fn branch( - &self, + self, condition: impl Fn(&S::Timestamp, &D) -> bool + 'static, ) -> (Stream, Stream) { let mut builder = OperatorBuilder::new("Branch".to_owned(), self.scope()); @@ -89,11 +89,11 @@ pub trait BranchWhen: Sized { /// after_five.inspect(|x| println!("Times 5 and later: {:?}", x)); /// }); /// ``` - fn branch_when(&self, condition: impl Fn(&T) -> bool + 'static) -> (Self, Self); + fn branch_when(self, condition: impl Fn(&T) -> bool + 'static) -> (Self, Self); } -impl BranchWhen for StreamCore { - fn branch_when(&self, condition: impl Fn(&S::Timestamp) -> bool + 'static) -> (Self, Self) { +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()); let mut input = builder.new_input(self, Pipeline); diff --git a/timely/src/dataflow/operators/broadcast.rs b/timely/src/dataflow/operators/broadcast.rs index 21d80d6da..8a1979d12 100644 --- a/timely/src/dataflow/operators/broadcast.rs +++ b/timely/src/dataflow/operators/broadcast.rs @@ -18,11 +18,11 @@ pub trait Broadcast { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn broadcast(&self) -> Self; + fn broadcast(self) -> Self; } -impl Broadcast for Stream { - fn broadcast(&self) -> Stream { +impl Broadcast for Stream { + fn broadcast(self) -> Stream { // NOTE: Simplified implementation due to underlying motion // in timely dataflow internals. Optimize once they have diff --git a/timely/src/dataflow/operators/core/capture/capture.rs b/timely/src/dataflow/operators/core/capture/capture.rs index 82bef229a..f6abe5346 100644 --- a/timely/src/dataflow/operators/core/capture/capture.rs +++ b/timely/src/dataflow/operators/core/capture/capture.rs @@ -10,14 +10,14 @@ use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; -use crate::{Container, Data}; +use crate::Container; use crate::progress::ChangeBatch; use crate::progress::Timestamp; use super::{Event, EventPusher}; /// Capture a stream of timestamped data for later replay. -pub trait Capture { +pub trait Capture : Sized { /// Captures a stream of timestamped data for later replay. /// /// # Examples @@ -103,18 +103,18 @@ pub trait Capture { /// /// assert_eq!(recv0.extract()[0].1, (0..10).collect::>()); /// ``` - fn capture_into+'static>(&self, pusher: P); + fn capture_into+'static>(self, pusher: P); /// Captures a stream using Rust's MPSC channels. - fn capture(&self) -> ::std::sync::mpsc::Receiver> { + fn capture(self) -> ::std::sync::mpsc::Receiver> { let (send, recv) = ::std::sync::mpsc::channel(); self.capture_into(send); recv } } -impl Capture for StreamCore { - fn capture_into+'static>(&self, mut event_pusher: P) { +impl Capture for StreamCore { + fn capture_into+'static>(self, mut event_pusher: P) { let mut builder = OperatorBuilder::new("Capture".to_owned(), self.scope()); let mut input = PullCounter::new(builder.new_input(self, Pipeline)); @@ -125,7 +125,7 @@ impl Capture for StreamCore { /// /// timely::example(|scope| { /// - /// let stream = (0..10).to_stream(scope); - /// stream.concat(&stream) + /// let stream = (0..10).to_stream(scope).tee(); + /// stream.owned().concat(stream.owned()) /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn concat(&self, _: &StreamCore) -> StreamCore; + fn concat(self, other: StreamCore) -> StreamCore; } -impl Concat for StreamCore { - fn concat(&self, other: &StreamCore) -> StreamCore { - self.scope().concatenate([self.clone(), other.clone()]) +impl Concat for StreamCore { + fn concat(self, other: StreamCore) -> StreamCore { + self.scope().concatenate([self, other]) } } @@ -43,27 +43,27 @@ pub trait Concatenate { /// (0..10).to_stream(scope), /// (0..10).to_stream(scope)]; /// - /// scope.concatenate(streams) + /// scope.clone() + /// .concatenate(streams) /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn concatenate(&self, sources: I) -> StreamCore + fn concatenate(self, sources: I) -> StreamCore where I: IntoIterator>; } -impl Concatenate for StreamCore { - fn concatenate(&self, sources: I) -> StreamCore +impl Concatenate for StreamCore { + fn concatenate(self, sources: I) -> StreamCore where I: IntoIterator> { - let clone = self.clone(); - self.scope().concatenate(Some(clone).into_iter().chain(sources)) + self.scope().concatenate(Some(self).into_iter().chain(sources)) } } -impl Concatenate for G { - fn concatenate(&self, sources: I) -> StreamCore +impl Concatenate for G { + fn concatenate(self, sources: I) -> StreamCore where I: IntoIterator> { @@ -73,7 +73,7 @@ impl Concatenate for G { let mut builder = OperatorBuilder::new("Concatenate".to_string(), self.clone()); // create new input handles for each input stream. - let mut handles = sources.into_iter().map(|s| builder.new_input(&s, Pipeline)).collect::>(); + 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(); diff --git a/timely/src/dataflow/operators/core/enterleave.rs b/timely/src/dataflow/operators/core/enterleave.rs index 16464a7f6..33d9a1453 100644 --- a/timely/src/dataflow/operators/core/enterleave.rs +++ b/timely/src/dataflow/operators/core/enterleave.rs @@ -25,9 +25,9 @@ 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::Container; use crate::communication::Push; -use crate::dataflow::channels::pushers::{Counter, Tee}; +use crate::dataflow::channels::pushers::{Counter, PushOwned}; use crate::dataflow::channels::Message; use crate::worker::AsWorker; @@ -50,15 +50,15 @@ pub trait Enter, C: Container> { /// }); /// }); /// ``` - fn enter<'a>(&self, _: &Child<'a, G, T>) -> StreamCore, C>; + fn enter<'a>(self, _: &Child<'a, G, T>) -> StreamCore, C>; } -impl, C: Data+Container> Enter for StreamCore { - fn enter<'a>(&self, scope: &Child<'a, G, T>) -> StreamCore, C> { +impl, C: Container+'static> Enter for StreamCore { + fn enter<'a>(self, scope: &Child<'a, G, T>) -> StreamCore, C> { use crate::scheduling::Scheduler; - let (targets, registrar) = Tee::::new(); + let (targets, registrar) = PushOwned::::new(); let ingress = IngressNub { targets: Counter::new(targets), phantom: PhantomData, @@ -100,20 +100,20 @@ pub trait Leave { /// }); /// }); /// ``` - fn leave(&self) -> StreamCore; + fn leave(self) -> StreamCore; } -impl> Leave for StreamCore, C> { - fn leave(&self) -> StreamCore { +impl> Leave for StreamCore, C> { + fn leave(self) -> StreamCore { let scope = self.scope(); let output = scope.subgraph.borrow_mut().new_output(); - let target = Target::new(0, output.port); - let (targets, registrar) = Tee::::new(); - let egress = EgressNub { targets, phantom: PhantomData }; + let (target, registrar) = PushOwned::::new(); + let egress = EgressNub { target, phantom: PhantomData }; let channel_id = scope.clone().new_identifier(); + let target = Target::new(0, output.port); if let Some(logger) = scope.logging() { let pusher = LogPusher::new(egress, channel_id, scope.index(), logger); self.connect_to(target, pusher, channel_id); @@ -130,14 +130,14 @@ impl> Leave, TContainer: Container + Data> { - targets: Counter>, - phantom: ::std::marker::PhantomData, +struct IngressNub, TContainer: Container> { + targets: Counter>, + phantom: PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TContainer: Container + Data> Push> for IngressNub { +impl, TContainer: Container> Push> for IngressNub { fn push(&mut self, element: &mut Option>) { if let Some(outer_message) = element { let data = ::std::mem::take(&mut outer_message.data); @@ -159,23 +159,23 @@ impl, TContainer: Container } -struct EgressNub, TContainer: Data> { - targets: Tee, +struct EgressNub, TContainer> { + target: PushOwned, phantom: PhantomData, } impl Push> for EgressNub -where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: Data { +where TOuter: Timestamp, TInner: Timestamp+Refines, TContainer: 'static { fn push(&mut self, message: &mut Option>) { if let Some(inner_message) = message { let data = ::std::mem::take(&mut inner_message.data); let mut outer_message = Some(Message::new(inner_message.time.clone().to_outer(), data, 0, 0)); - self.targets.push(&mut outer_message); + self.target.push(&mut outer_message); if let Some(outer_message) = outer_message { inner_message.data = outer_message.data; } } - else { self.targets.done(); } + else { self.target.done(); } } } diff --git a/timely/src/dataflow/operators/core/exchange.rs b/timely/src/dataflow/operators/core/exchange.rs index bb021a5d5..403d10639 100644 --- a/timely/src/dataflow/operators/core/exchange.rs +++ b/timely/src/dataflow/operators/core/exchange.rs @@ -23,7 +23,7 @@ pub trait Exchange { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn exchange(&self, route: F) -> Self + fn exchange(self, route: F) -> Self where for<'a> F: FnMut(&C::Item<'a>) -> u64 + 'static; } @@ -32,9 +32,8 @@ impl Exchange for StreamCore where C: SizableContainer + ExchangeData + crate::dataflow::channels::ContainerBytes, C: for<'a> PushInto>, - { - fn exchange(&self, route: F) -> StreamCore + fn exchange(self, route: F) -> StreamCore where for<'a> F: FnMut(&C::Item<'a>) -> u64 + 'static, { diff --git a/timely/src/dataflow/operators/core/feedback.rs b/timely/src/dataflow/operators/core/feedback.rs index e008b3029..a2a296b78 100644 --- a/timely/src/dataflow/operators/core/feedback.rs +++ b/timely/src/dataflow/operators/core/feedback.rs @@ -1,9 +1,9 @@ //! Create cycles in a timely dataflow graph. -use crate::{Container, Data}; +use crate::Container; use crate::container::CapacityContainerBuilder; use crate::dataflow::channels::pact::Pipeline; -use crate::dataflow::channels::pushers::Tee; +use crate::dataflow::channels::pushers::PushOwned; use crate::dataflow::operators::generic::OutputWrapper; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::scopes::child::Iterative; @@ -30,13 +30,13 @@ pub trait Feedback { /// // circulate 0..10 for 100 iterations. /// let (handle, cycle) = scope.feedback(1); /// (0..10).to_stream(scope) - /// .concat(&cycle) + /// .concat(cycle) /// .inspect(|x| println!("seen: {:?}", x)) /// .branch_when(|t| t < &100).1 /// .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`. @@ -57,19 +57,19 @@ pub trait LoopVariable<'a, G: Scope, T: Timestamp> { /// scope.iterative::(|inner| { /// let (handle, cycle) = inner.loop_variable(1); /// (0..10).to_stream(inner) - /// .concat(&cycle) + /// .concat(cycle) /// .inspect(|x| println!("seen: {:?}", x)) /// .branch_when(|t| t.inner < 100).1 /// .connect_loop(handle); /// }); /// }); /// ``` - 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()); let (output, stream) = builder.new_output(); @@ -79,13 +79,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 @@ -97,17 +97,17 @@ pub trait ConnectLoop { /// // circulate 0..10 for 100 iterations. /// let (handle, cycle) = scope.feedback(1); /// (0..10).to_stream(scope) - /// .concat(&cycle) + /// .concat(cycle) /// .inspect(|x| println!("seen: {:?}", x)) /// .branch_when(|t| t < &100).1 /// .connect_loop(handle); /// }); /// ``` - fn connect_loop(&self, handle: Handle); + fn connect_loop(self, handle: Handle); } -impl ConnectLoop for StreamCore { - fn connect_loop(&self, handle: Handle) { +impl ConnectLoop for StreamCore { + fn connect_loop(self, handle: Handle) { let mut builder = handle.builder; let summary = handle.summary; @@ -131,8 +131,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, PushOwned>, } diff --git a/timely/src/dataflow/operators/core/filter.rs b/timely/src/dataflow/operators/core/filter.rs index 80ef564b0..94ae71273 100644 --- a/timely/src/dataflow/operators/core/filter.rs +++ b/timely/src/dataflow/operators/core/filter.rs @@ -1,6 +1,5 @@ //! Filters a stream by a predicate. use crate::container::{Container, SizableContainer, PushInto}; -use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::operators::generic::operator::Operator; @@ -20,14 +19,14 @@ pub trait Filter { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn filter)->bool+'static>(&self, predicate: P) -> Self; + fn filter)->bool+'static>(self, predicate: P) -> Self; } -impl Filter for StreamCore +impl Filter for StreamCore where for<'a> C: PushInto> { - fn filter)->bool+'static>(&self, mut predicate: P) -> StreamCore { + fn filter)->bool+'static>(self, mut predicate: P) -> StreamCore { self.unary(Pipeline, "Filter", move |_,_| move |input, output| { input.for_each(|time, data| { if !data.is_empty() { diff --git a/timely/src/dataflow/operators/core/input.rs b/timely/src/dataflow/operators/core/input.rs index a1af533c2..21ecdf703 100644 --- a/timely/src/dataflow/operators/core/input.rs +++ b/timely/src/dataflow/operators/core/input.rs @@ -11,10 +11,10 @@ use crate::progress::frontier::Antichain; use crate::progress::{Operate, operate::SharedProgress, Timestamp, ChangeBatch}; use crate::progress::Source; -use crate::{Container, Data}; +use crate::Container; use crate::communication::Push; use crate::dataflow::{Scope, ScopeParent, StreamCore}; -use crate::dataflow::channels::pushers::{Tee, Counter}; +use crate::dataflow::channels::pushers::{Counter, PushOwned}; use crate::dataflow::channels::Message; @@ -60,7 +60,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. /// @@ -97,7 +97,10 @@ pub trait Input : Scope { /// } /// }); /// ``` - fn new_input_with_builder(&mut self) -> (Handle<::Timestamp, CB>, StreamCore); + fn new_input_with_builder(&mut self) -> (Handle<::Timestamp, CB>, StreamCore) + where + CB::Container: Clone, + ; /// Create a new stream from a supplied interactive handle. /// @@ -130,25 +133,34 @@ pub trait Input : Scope { /// } /// }); /// ``` - fn input_from(&mut self, handle: &mut Handle<::Timestamp, CB>) -> StreamCore; + fn input_from(&mut self, handle: &mut Handle<::Timestamp, CB>) -> StreamCore + where + CB::Container: Clone, + ; } 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) } - fn new_input_with_builder(&mut self) -> (Handle<::Timestamp, CB>, StreamCore) { + fn new_input_with_builder(&mut self) -> (Handle<::Timestamp, CB>, StreamCore) + where + CB::Container: Clone, + { let mut handle = Handle::new_with_builder(); let stream = self.input_from(&mut handle); (handle, stream) } - fn input_from(&mut self, handle: &mut Handle<::Timestamp, CB>) -> StreamCore { - let (output, registrar) = Tee::<::Timestamp, CB::Container>::new(); + fn input_from(&mut self, handle: &mut Handle<::Timestamp, CB>) -> StreamCore + where + CB::Container: Clone, + { + let (output, registrar) = PushOwned::<::Timestamp, CB::Container>::new(); let counter = Counter::new(output); let produced = counter.produced().clone(); @@ -216,16 +228,19 @@ impl Operate for Operator { /// A handle to an input `StreamCore`, used to introduce data to a timely dataflow computation. #[derive(Debug)] -pub struct Handle { +pub struct Handle +where + CB::Container: Clone, +{ activate: Vec, progress: Vec>>>, - pushers: Vec>>, + pushers: Vec>>, builder: CB, buffer: CB::Container, now_at: T, } -impl Handle> { +impl Handle> { /// Allocates a new input handle, from which one can create timely streams. /// /// # Examples @@ -265,7 +280,10 @@ impl Handle> { } } -impl Handle { +impl Handle +where + CB::Container: Clone, +{ /// Allocates a new input handle, from which one can create timely streams. /// /// # Examples @@ -342,7 +360,7 @@ impl Handle { fn register( &mut self, - pusher: Counter>, + pusher: Counter>, progress: Rc>>, ) { // flush current contents, so new registrant does not see existing data. @@ -381,7 +399,7 @@ impl Handle { fn send_container( container: &mut CB::Container, buffer: &mut CB::Container, - pushers: &mut [Counter>], + pushers: &mut [Counter>], now_at: &T ) { for index in 0 .. pushers.len() { @@ -486,6 +504,7 @@ impl PushInto for Handle where T: Timestamp, CB: ContainerBuilder + PushInto, + CB::Container: Clone, { #[inline] fn push_into(&mut self, item: D) { @@ -494,7 +513,10 @@ where } } -impl Handle { +impl Handle +where + CB::Container: Clone, +{ /// Sends one record into the corresponding timely dataflow `Stream`, at the current epoch. /// /// # Examples @@ -528,13 +550,19 @@ impl Handle { } } -impl Default for Handle { +impl Default for Handle +where + CB::Container: Clone, +{ fn default() -> Self { Self::new_with_builder() } } -impl Drop for Handle { +impl Drop for Handle +where + CB::Container: Clone, +{ fn drop(&mut self) { self.close_epoch(); } diff --git a/timely/src/dataflow/operators/core/inspect.rs b/timely/src/dataflow/operators/core/inspect.rs index b8c41f97b..5e28eab55 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, Data}; +use crate::Container; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::operators::generic::Operator; @@ -18,7 +18,7 @@ pub trait Inspect: InspectCore + Sized { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn inspect(&self, mut func: F) -> Self + fn inspect(self, mut func: F) -> Self where F: for<'a> FnMut(C::ItemRef<'a>) + 'static, { @@ -38,7 +38,7 @@ pub trait Inspect: InspectCore + Sized { /// .inspect_time(|t, x| println!("seen at: {:?}\t{:?}", t, x)); /// }); /// ``` - fn inspect_time(&self, mut func: F) -> Self + fn inspect_time(self, mut func: F) -> Self where F: for<'a> FnMut(&G::Timestamp, C::ItemRef<'a>) + 'static, { @@ -60,7 +60,7 @@ pub trait Inspect: InspectCore + Sized { /// .inspect_batch(|t,xs| println!("seen at: {:?}\t{:?} records", t, xs.len())); /// }); /// ``` - fn inspect_batch(&self, mut func: impl FnMut(&G::Timestamp, &C)+'static) -> Self { + fn inspect_batch(self, mut func: impl FnMut(&G::Timestamp, &C)+'static) -> Self { self.inspect_core(move |event| { if let Ok((time, data)) = event { func(time, data); @@ -87,11 +87,11 @@ pub trait Inspect: InspectCore + Sized { /// }); /// }); /// ``` - fn inspect_core(&self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; + fn inspect_core(self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; } -impl Inspect for StreamCore { - fn inspect_core(&self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>) + 'static { +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) } } @@ -117,12 +117,12 @@ pub trait InspectCore { /// }); /// }); /// ``` - fn inspect_container(&self, func: F) -> StreamCore where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; + fn inspect_container(self, func: F) -> Self 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 + fn inspect_container(self, mut func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static { use crate::progress::timestamp::Timestamp; diff --git a/timely/src/dataflow/operators/core/map.rs b/timely/src/dataflow/operators/core/map.rs index 8af70e4a4..4b5bd7e6f 100644 --- a/timely/src/dataflow/operators/core/map.rs +++ b/timely/src/dataflow/operators/core/map.rs @@ -1,13 +1,12 @@ //! Extension methods for `StreamCore` based on record-by-record transformation. use crate::container::{Container, 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 : Sized { /// Consumes each element of the stream and yields a new element. /// /// # Examples @@ -22,9 +21,9 @@ pub trait Map { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn map(&self, mut logic: L) -> StreamCore + fn map(self, mut logic: L) -> StreamCore where - C2: SizableContainer + PushInto + Data, + C2: SizableContainer + PushInto + 'static, L: FnMut(C::Item<'_>)->D2 + 'static, { self.flat_map(move |x| std::iter::once(logic(x))) @@ -43,22 +42,22 @@ pub trait Map { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn flat_map(&self, logic: L) -> StreamCore + fn flat_map(self, logic: L) -> StreamCore where I: IntoIterator, - C2: SizableContainer + PushInto + Data, + C2: SizableContainer + PushInto + 'static, 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 + fn flat_map(self, mut logic: L) -> StreamCore where I: IntoIterator, - C2: SizableContainer + PushInto + Data, + C2: SizableContainer + PushInto + 'static, 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 fd7887053..1b6aa74b1 100644 --- a/timely/src/dataflow/operators/core/ok_err.rs +++ b/timely/src/dataflow/operators/core/ok_err.rs @@ -1,7 +1,6 @@ //! Operators that separate one stream into two streams based on some condition use crate::container::{Container, SizableContainer, PushInto}; -use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, StreamCore}; @@ -29,24 +28,24 @@ pub trait OkErr { /// }); /// ``` fn ok_err( - &self, + self, logic: L, ) -> (StreamCore, StreamCore) where - C1: SizableContainer + PushInto + Data, - C2: SizableContainer + PushInto + Data, + C1: SizableContainer + PushInto + 'static, + C2: SizableContainer + PushInto + 'static, L: FnMut(C::Item<'_>) -> Result+'static ; } -impl OkErr for StreamCore { +impl OkErr for StreamCore { fn ok_err( - &self, + self, mut logic: L, ) -> (StreamCore, StreamCore) where - C1: SizableContainer + PushInto + Data, - C2: SizableContainer + PushInto + Data, + C1: SizableContainer + PushInto + 'static, + C2: SizableContainer + PushInto + 'static, L: FnMut(C::Item<'_>) -> Result+'static { let mut builder = OperatorBuilder::new("OkErr".to_owned(), self.scope()); diff --git a/timely/src/dataflow/operators/core/probe.rs b/timely/src/dataflow/operators/core/probe.rs index c7fdced33..49909a457 100644 --- a/timely/src/dataflow/operators/core/probe.rs +++ b/timely/src/dataflow/operators/core/probe.rs @@ -10,10 +10,8 @@ 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::Container; /// Monitors progress at a `Stream`. pub trait Probe { @@ -44,7 +42,7 @@ pub trait Probe { /// } /// }).unwrap(); /// ``` - fn probe(&self) -> Handle; + fn probe(self) -> Handle; /// Inserts a progress probe in a stream. /// @@ -76,18 +74,18 @@ pub trait Probe { /// } /// }).unwrap(); /// ``` - fn probe_with(&self, handle: &Handle) -> StreamCore; + fn probe_with(self, handle: &Handle) -> StreamCore; } -impl Probe for StreamCore { - fn probe(&self) -> Handle { +impl Probe for StreamCore { + fn probe(self) -> Handle { // the frontier is shared state; scope updates, handle reads. let handle = Handle::::new(); self.probe_with(&handle); handle } - fn probe_with(&self, handle: &Handle) -> StreamCore { + fn probe_with(self, handle: &Handle) -> StreamCore { let mut builder = OperatorBuilder::new("Probe".to_owned(), self.scope()); let mut input = PullCounter::new(builder.new_input(self, Pipeline)); diff --git a/timely/src/dataflow/operators/core/rc.rs b/timely/src/dataflow/operators/core/rc.rs index fdc68b9d4..b90d52493 100644 --- a/timely/src/dataflow/operators/core/rc.rs +++ b/timely/src/dataflow/operators/core/rc.rs @@ -1,10 +1,10 @@ //! Shared containers +use std::rc::Rc; 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::Container; /// Convert a stream into a stream of shared containers pub trait SharedStream { @@ -21,11 +21,11 @@ pub trait SharedStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn shared(&self) -> StreamCore>; + fn shared(self) -> StreamCore>; } -impl SharedStream for StreamCore { - fn shared(&self) -> StreamCore> { +impl SharedStream for StreamCore { + fn shared(self) -> StreamCore> { self.unary(Pipeline, "Shared", move |_, _| { move |input, output| { input.for_each(|time, data| { @@ -48,17 +48,18 @@ mod test { #[test] fn test_shared() { let output = crate::example(|scope| { - let shared = vec![Ok(0), Err(())].to_stream(scope).container::>().shared(); + let shared = vec![Ok(0), Err(())].to_stream(scope).container::>().shared().tee(); scope + .clone() .concatenate([ - shared.unary(Pipeline, "read shared 1", |_, _| { + shared.owned().unary(Pipeline, "read shared 1", |_, _| { move |input, output| { input.for_each(|time, data| { output.session(&time).give(data.as_ptr() as usize); }); } }), - shared.unary(Pipeline, "read shared 2", |_, _| { + shared.owned().unary(Pipeline, "read shared 2", |_, _| { move |input, output| { input.for_each(|time, data| { output.session(&time).give(data.as_ptr() as usize); diff --git a/timely/src/dataflow/operators/core/reclock.rs b/timely/src/dataflow/operators/core/reclock.rs index e74bedbc4..cd1bf39d6 100644 --- a/timely/src/dataflow/operators/core/reclock.rs +++ b/timely/src/dataflow/operators/core/reclock.rs @@ -1,6 +1,6 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::{Container, Data}; +use crate::Container; use crate::order::PartialOrder; use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; @@ -35,7 +35,7 @@ pub trait Reclock { /// .map(|_| ()); /// /// // reclock the data. - /// data.reclock(&clock) + /// data.reclock(clock) /// .capture() /// }); /// @@ -45,11 +45,11 @@ pub trait Reclock { /// assert_eq!(extracted[1], (5, vec![4,5])); /// assert_eq!(extracted[2], (8, vec![6,7,8])); /// ``` - fn reclock(&self, clock: &StreamCore) -> Self; + fn reclock(self, clock: StreamCore) -> Self; } -impl Reclock for StreamCore { - fn reclock(&self, clock: &StreamCore) -> StreamCore { +impl Reclock for StreamCore { + fn reclock(self, clock: StreamCore) -> StreamCore { let mut stash = vec![]; diff --git a/timely/src/dataflow/operators/core/to_stream.rs b/timely/src/dataflow/operators/core/to_stream.rs index cfc6f429b..0a6fcf0eb 100644 --- a/timely/src/dataflow/operators/core/to_stream.rs +++ b/timely/src/dataflow/operators/core/to_stream.rs @@ -1,7 +1,7 @@ //! Conversion to the `StreamCore` type from iterators. use crate::container::{CapacityContainerBuilder, ContainerBuilder, SizableContainer, PushInto}; -use crate::{Container, Data}; +use crate::Container; use crate::dataflow::operators::generic::operator::source; use crate::dataflow::{StreamCore, Scope}; @@ -81,7 +81,7 @@ pub trait ToStream { fn to_stream(self, scope: &mut S) -> StreamCore; } -impl ToStream for I where C: PushInto { +impl ToStream for I where C: PushInto { fn to_stream(self, scope: &mut S) -> StreamCore { ToStreamBuilder::>::to_stream_with_builder(self, scope) } diff --git a/timely/src/dataflow/operators/core/unordered_input.rs b/timely/src/dataflow/operators/core/unordered_input.rs index aacd89ca9..a0f457098 100644 --- a/timely/src/dataflow/operators/core/unordered_input.rs +++ b/timely/src/dataflow/operators/core/unordered_input.rs @@ -2,7 +2,7 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::{Container, Data}; +use crate::Container; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; use crate::scheduling::{Schedule, ActivateOnDrop}; @@ -12,7 +12,7 @@ use crate::progress::{Operate, operate::SharedProgress, Timestamp}; use crate::progress::Source; use crate::progress::ChangeBatch; -use crate::dataflow::channels::pushers::{Counter, Tee}; +use crate::dataflow::channels::pushers::{Counter, PushOwned}; use crate::dataflow::channels::pushers::buffer::{Buffer as PushBuffer, AutoflushSession}; use crate::dataflow::operators::{ActivateCapability, Capability}; @@ -82,7 +82,7 @@ pub trait UnorderedInput { impl UnorderedInput for G { fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), StreamCore) { - let (output, registrar) = Tee::::new(); + let (output, registrar) = PushOwned::::new(); let internal = Rc::new(RefCell::new(ChangeBatch::new())); // let produced = Rc::new(RefCell::new(ChangeBatch::new())); let cap = Capability::new(G::Timestamp::minimum(), internal.clone()); @@ -148,11 +148,11 @@ impl Operate for UnorderedOperator { /// A handle to an input [StreamCore], used to introduce data to a timely dataflow computation. #[derive(Debug)] pub struct UnorderedHandle { - buffer: PushBuffer>>, + buffer: PushBuffer>>, } impl UnorderedHandle { - fn new(pusher: Counter>) -> UnorderedHandle { + fn new(pusher: Counter>) -> UnorderedHandle { UnorderedHandle { buffer: PushBuffer::new(pusher), } @@ -160,15 +160,15 @@ impl UnorderedHandle { /// Allocates a new automatically flushing session based on the supplied capability. #[inline] - pub fn session_with_builder(&mut self, cap: ActivateCapability) -> ActivateOnDrop>>> { + pub fn session_with_builder(&mut self, cap: ActivateCapability) -> ActivateOnDrop>>> { ActivateOnDrop::new(self.buffer.autoflush_session_with_builder(cap.capability.clone()), cap.address.clone(), cap.activations.clone()) } } -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>>> { + pub fn session(&mut self, cap: ActivateCapability) -> ActivateOnDrop, Counter>>> { self.session_with_builder(cap) } } diff --git a/timely/src/dataflow/operators/count.rs b/timely/src/dataflow/operators/count.rs index 89e650b2b..9469bd81c 100644 --- a/timely/src/dataflow/operators/count.rs +++ b/timely/src/dataflow/operators/count.rs @@ -1,13 +1,12 @@ //! Counts the number of records at each time. use std::collections::HashMap; -use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Stream, Scope}; use crate::dataflow::operators::generic::operator::Operator; /// Accumulates records within a timestamp. -pub trait Accumulate { +pub trait Accumulate: Sized { /// Accumulates records within a timestamp. /// /// # Examples @@ -25,7 +24,7 @@ pub trait Accumulate { /// let extracted = captured.extract(); /// assert_eq!(extracted, vec![(0, vec![45])]); /// ``` - fn accumulate(&self, default: A, logic: impl Fn(&mut A, &mut Vec)+'static) -> Stream; + fn accumulate(self, default: A, logic: impl Fn(&mut A, &mut Vec)+'static) -> Stream; /// Counts the number of records observed at each time. /// /// # Examples @@ -43,13 +42,13 @@ pub trait Accumulate { /// let extracted = captured.extract(); /// assert_eq!(extracted, vec![(0, vec![10])]); /// ``` - fn count(&self) -> Stream { + fn count(self) -> Stream { self.accumulate(0, |sum, data| *sum += data.len()) } } -impl Accumulate for Stream { - fn accumulate(&self, default: A, logic: impl Fn(&mut A, &mut Vec)+'static) -> Stream { +impl Accumulate for Stream { + fn accumulate(self, default: A, logic: impl Fn(&mut A, &mut Vec)+'static) -> Stream { let mut accums = HashMap::new(); self.unary_notify(Pipeline, "Accumulate", vec![], move |input, output, notificator| { diff --git a/timely/src/dataflow/operators/delay.rs b/timely/src/dataflow/operators/delay.rs index 8638bfd08..63785e74c 100644 --- a/timely/src/dataflow/operators/delay.rs +++ b/timely/src/dataflow/operators/delay.rs @@ -2,14 +2,13 @@ use std::collections::HashMap; -use crate::Data; use crate::order::{PartialOrder, TotalOrder}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Stream, Scope}; use crate::dataflow::operators::generic::operator::Operator; /// Methods to advance the timestamps of records or batches of records. -pub trait Delay { +pub trait Delay { /// Advances the timestamp of records using a supplied function. /// @@ -36,7 +35,7 @@ pub trait Delay { /// }); /// }); /// ``` - fn delayG::Timestamp+'static>(&self, func: L) -> Self; + fn delayG::Timestamp+'static>(self, func: L) -> Self; /// Advances the timestamp of records using a supplied function. /// @@ -63,7 +62,7 @@ pub trait Delay { /// }); /// }); /// ``` - fn delay_totalG::Timestamp+'static>(&self, func: L) -> Self + fn delay_totalG::Timestamp+'static>(self, func: L) -> Self where G::Timestamp: TotalOrder; /// Advances the timestamp of batches of records using a supplied function. @@ -91,11 +90,11 @@ pub trait Delay { /// }); /// }); /// ``` - fn delay_batchG::Timestamp+'static>(&self, func: L) -> Self; + fn delay_batchG::Timestamp+'static>(self, func: L) -> Self; } -impl Delay for Stream { - fn delayG::Timestamp+'static>(&self, mut func: L) -> Self { +impl Delay for Stream { + fn delayG::Timestamp+'static>(self, mut func: L) -> Self { let mut elements = HashMap::new(); self.unary_notify(Pipeline, "Delay", vec![], move |input, output, notificator| { input.for_each(|time, data| { @@ -117,13 +116,13 @@ impl Delay for Stream { }) } - fn delay_totalG::Timestamp+'static>(&self, func: L) -> Self + fn delay_totalG::Timestamp+'static>(self, func: L) -> Self where G::Timestamp: TotalOrder { self.delay(func) } - fn delay_batchG::Timestamp+'static>(&self, mut func: L) -> Self { + fn delay_batchG::Timestamp+'static>(self, mut func: L) -> Self { let mut elements = HashMap::new(); self.unary_notify(Pipeline, "Delay", vec![], move |input, output, notificator| { input.for_each(|time, data| { diff --git a/timely/src/dataflow/operators/filter.rs b/timely/src/dataflow/operators/filter.rs index c2486e9eb..ff7c4259f 100644 --- a/timely/src/dataflow/operators/filter.rs +++ b/timely/src/dataflow/operators/filter.rs @@ -1,12 +1,11 @@ //! Filters a stream by a predicate. -use crate::Data; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::{Stream, Scope}; 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 @@ -19,11 +18,11 @@ pub trait Filter { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn filterbool+'static>(&self, predicate: P) -> Self; + fn filterbool+'static>(self, predicate: P) -> Self; } -impl Filter for Stream { - fn filterbool+'static>(&self, mut predicate: P) -> Stream { +impl Filter for Stream { + fn filterbool+'static>(self, mut predicate: P) -> Stream { self.unary(Pipeline, "Filter", move |_,_| move |input, output| { input.for_each(|time, data| { data.retain(|x| predicate(x)); diff --git a/timely/src/dataflow/operators/flow_controlled.rs b/timely/src/dataflow/operators/flow_controlled.rs index 0d91ef7f9..814bfbc00 100644 --- a/timely/src/dataflow/operators/flow_controlled.rs +++ b/timely/src/dataflow/operators/flow_controlled.rs @@ -1,6 +1,5 @@ //! Methods to construct flow-controlled sources. -use crate::Data; use crate::order::{PartialOrder, TotalOrder}; use crate::progress::timestamp::Timestamp; use crate::dataflow::operators::generic::operator::source; @@ -8,7 +7,7 @@ use crate::dataflow::operators::probe::Handle; use crate::dataflow::{Stream, Scope}; /// Output of the input reading function for iterator_source. -pub struct IteratorSourceInput, I: IntoIterator> { +pub struct IteratorSourceInput, I: IntoIterator> { /// Lower bound on timestamps that can be emitted by this input in the future. pub lower_bound: T, /// Any `T: IntoIterator` of new input data in the form (time, data): time must be @@ -73,7 +72,7 @@ pub struct IteratorSourceInput, I: I /// ``` pub fn iterator_source< G: Scope, - D: Data, + D: 'static, DI: IntoIterator, I: IntoIterator, F: FnMut(&G::Timestamp)->Option>+'static>( diff --git a/timely/src/dataflow/operators/generic/builder_raw.rs b/timely/src/dataflow/operators/generic/builder_raw.rs index 8ba131612..da91517af 100644 --- a/timely/src/dataflow/operators/generic/builder_raw.rs +++ b/timely/src/dataflow/operators/generic/builder_raw.rs @@ -15,8 +15,8 @@ use crate::progress::{Timestamp, Operate, operate::SharedProgress, Antichain}; use crate::Container; use crate::dataflow::{StreamCore, Scope}; -use crate::dataflow::channels::pushers::Tee; use crate::dataflow::channels::pact::ParallelizationContract; +use crate::dataflow::channels::pushers::PushOwned; use crate::dataflow::operators::generic::operator_info::OperatorInfo; /// Contains type-free information about the operator properties. @@ -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 { let connection = vec![Antichain::from_elem(Default::default()); self.shape.outputs]; @@ -112,7 +112,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: Vec::Summary>>) -> P::Puller + pub fn new_input_connection(&mut self, stream: StreamCore, pact: P, connection: Vec::Summary>>) -> P::Puller where P: ParallelizationContract { @@ -130,16 +130,16 @@ 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) -> (PushOwned, StreamCore) { let connection = vec![Antichain::from_elem(Default::default()); self.shape.inputs]; 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: Vec::Summary>>) -> (Tee, StreamCore) { + pub fn new_output_connection(&mut self, connection: Vec::Summary>>) -> (PushOwned, StreamCore) { - let (targets, registrar) = Tee::::new(); + let (target, registrar) = PushOwned::new(); let source = Source::new(self.index, self.shape.outputs); let stream = StreamCore::new(source, registrar, self.scope.clone()); @@ -149,7 +149,7 @@ impl OperatorBuilder { summary.push(entry); } - (targets, stream) + (target, stream) } /// Creates an operator implementation from supplied logic constructor. diff --git a/timely/src/dataflow/operators/generic/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index 752b3be2e..e20c594e0 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -11,11 +11,11 @@ use crate::progress::frontier::{Antichain, MutableAntichain}; 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; use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; use crate::dataflow::channels::pact::ParallelizationContract; use crate::dataflow::channels::pullers::Counter as PullCounter; +use crate::dataflow::channels::pushers::PushOwned; use crate::dataflow::operators::capability::Capability; use crate::dataflow::operators::generic::handles::{InputHandleCore, new_input_handle, OutputWrapper}; use crate::dataflow::operators::generic::operator_info::OperatorInfo; @@ -60,10 +60,10 @@ 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 { - + P: ParallelizationContract + { let connection = (0..self.builder.shape().outputs()).map(|_| Antichain::from_elem(Default::default())).collect(); self.new_input_connection(stream, pact, connection) } @@ -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: Vec::Summary>>) -> InputHandleCore + pub fn new_input_connection(&mut self, stream: StreamCore, pact: P, connection: Vec::Summary>>) -> InputHandleCore where P: ParallelizationContract { @@ -93,8 +93,8 @@ impl OperatorBuilder { } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (OutputWrapper>, StreamCore) { - let connection = (0..self.builder.shape().inputs()).map(|_| Antichain::from_elem(Default::default())).collect(); + pub fn new_output(&mut self) -> (OutputWrapper>, StreamCore) { + let connection = vec![Antichain::from_elem(Default::default()); self.builder.shape().inputs()]; self.new_output_connection(connection) } @@ -110,7 +110,7 @@ impl OperatorBuilder { &mut self, connection: Vec::Summary>> ) -> ( - OutputWrapper>, + OutputWrapper>, StreamCore ) { diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index 47811a659..b147d8bbf 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -15,7 +15,7 @@ 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; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; use crate::logging::TimelyLogger as Logger; @@ -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: Container + 'static, P: Push>> OutputHandleCore<'a, T, CapacityContainerBuilder, P> { /// Obtains a session that can send data at the timestamp associated with capability `cap`. /// /// In order to send data at a future timestamp, obtain a capability for the new timestamp diff --git a/timely/src/dataflow/operators/generic/notificator.rs b/timely/src/dataflow/operators/generic/notificator.rs index 905fdef4f..900d136b5 100644 --- a/timely/src/dataflow/operators/generic/notificator.rs +++ b/timely/src/dataflow/operators/generic/notificator.rs @@ -195,7 +195,7 @@ fn notificator_delivers_notifications_in_topo_order() { /// let (mut in1, mut in2) = worker.dataflow::(|scope| { /// let (in1_handle, in1) = scope.new_input(); /// let (in2_handle, in2) = scope.new_input(); -/// in1.binary_frontier(&in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { +/// in1.binary_frontier(in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { /// let mut notificator = FrontierNotificator::default(); /// let mut stash = HashMap::new(); /// move |input1, input2, output| { diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index 247cb5258..300afddf6 100644 --- a/timely/src/dataflow/operators/generic/operator.rs +++ b/timely/src/dataflow/operators/generic/operator.rs @@ -1,7 +1,7 @@ //! Methods to construct generic streaming and blocking unary operators. -use crate::dataflow::channels::pushers::Tee; +use crate::dataflow::channels::pushers::PushOwned; use crate::dataflow::channels::pact::ParallelizationContract; use crate::dataflow::operators::generic::handles::{InputHandleCore, FrontieredInputHandleCore, OutputHandleCore}; @@ -12,7 +12,7 @@ use crate::dataflow::{Scope, StreamCore}; use super::builder_rc::OperatorBuilder; use crate::dataflow::operators::generic::OperatorInfo; use crate::dataflow::operators::generic::notificator::{Notificator, FrontierNotificator}; -use crate::{Container, Data}; +use crate::Container; use crate::container::{ContainerBuilder, CapacityContainerBuilder}; /// Methods to construct generic streaming and blocking operators. @@ -55,12 +55,12 @@ pub trait Operator { /// }); /// } /// ``` - fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> StreamCore + fn unary_frontier(self, pact: P, name: &str, constructor: B) -> StreamCore where CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P: ParallelizationContract; /// Creates a new dataflow operator that partitions its input stream by a parallelization @@ -92,10 +92,10 @@ pub trait Operator { /// ``` fn unary_notify, - &mut OutputHandleCore>, + &mut OutputHandleCore>, &mut Notificator)+'static, P: ParallelizationContract> - (&self, pact: P, name: &str, init: impl IntoIterator, logic: L) -> StreamCore; + (self, pact: P, name: &str, init: impl IntoIterator, logic: L) -> StreamCore; /// 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`. @@ -123,12 +123,12 @@ pub trait Operator { /// }); /// }); /// ``` - fn unary(&self, pact: P, name: &str, constructor: B) -> StreamCore + fn unary(self, pact: P, name: &str, constructor: B) -> StreamCore where CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P: ParallelizationContract; /// Creates a new dataflow operator that partitions its input streams by a parallelization @@ -146,7 +146,7 @@ pub trait Operator { /// let (mut in1, mut in2) = worker.dataflow::(|scope| { /// let (in1_handle, in1) = scope.new_input(); /// let (in2_handle, in2) = scope.new_input(); - /// in1.binary_frontier(&in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { + /// in1.binary_frontier(in2, Pipeline, Pipeline, "example", |mut _default_cap, _info| { /// let mut notificator = FrontierNotificator::default(); /// let mut stash = HashMap::new(); /// move |input1, input2, output| { @@ -179,14 +179,14 @@ pub trait Operator { /// } /// }).unwrap(); /// ``` - fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore + fn binary_frontier(self, other: StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container + 'static, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, &mut FrontieredInputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P1: ParallelizationContract, P2: ParallelizationContract; @@ -206,7 +206,7 @@ pub trait Operator { /// let (in1_handle, in1) = scope.new_input(); /// let (in2_handle, in2) = scope.new_input(); /// - /// in1.binary_notify(&in2, Pipeline, Pipeline, "example", None, move |input1, input2, output, notificator| { + /// in1.binary_notify(in2, Pipeline, Pipeline, "example", None, move |input1, input2, output, notificator| { /// input1.for_each(|time, data| { /// output.session(&time).give_container(data); /// notificator.notify_at(time.retain()); @@ -231,15 +231,15 @@ pub trait Operator { /// } /// }).unwrap(); /// ``` - fn binary_notify, &mut InputHandleCore, - &mut OutputHandleCore>, + &mut OutputHandleCore>, &mut Notificator)+'static, P1: ParallelizationContract, P2: ParallelizationContract> - (&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, logic: L) -> StreamCore; + (self, other: StreamCore, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, logic: L) -> StreamCore; /// Creates a new dataflow operator that partitions its input streams by a parallelization /// strategy `pact`, and repeatedly invokes `logic`, the function returned by the function passed as `constructor`. @@ -255,7 +255,7 @@ pub trait Operator { /// timely::example(|scope| { /// let stream2 = (0u64..10).to_stream(scope); /// (0u64..10).to_stream(scope) - /// .binary(&stream2, Pipeline, Pipeline, "example", |default_cap, _info| { + /// .binary(stream2, Pipeline, Pipeline, "example", |default_cap, _info| { /// let mut cap = Some(default_cap.delayed(&12)); /// move |input1, input2, output| { /// if let Some(ref c) = cap.take() { @@ -271,14 +271,14 @@ pub trait Operator { /// }).inspect(|x| println!("{:?}", x)); /// }); /// ``` - fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore + fn binary(self, other: StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container + 'static, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, &mut InputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P1: ParallelizationContract, P2: ParallelizationContract; @@ -305,20 +305,20 @@ pub trait Operator { /// }); /// }); /// ``` - fn sink(&self, pact: P, name: &str, logic: L) + fn sink(self, pact: P, name: &str, logic: L) where L: FnMut(&mut FrontieredInputHandleCore)+'static, P: ParallelizationContract; } -impl Operator for StreamCore { +impl Operator for StreamCore { - fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> StreamCore + fn unary_frontier(self, pact: P, name: &str, constructor: B) -> StreamCore where CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P: ParallelizationContract { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); @@ -343,18 +343,18 @@ impl Operator for StreamCore { fn unary_notify, - &mut OutputHandleCore>, + &mut OutputHandleCore>, &mut Notificator)+'static, P: ParallelizationContract> - (&self, pact: P, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { + (self, pact: P, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { + let logging = self.scope().logging(); self.unary_frontier(pact, name, move |capability, _info| { let mut notificator = FrontierNotificator::default(); for time in init { notificator.notify_at(capability.delayed(&time)); } - let logging = self.scope().logging(); move |input, output| { let frontier = &[input.frontier()]; let notificator = &mut Notificator::new(frontier, &mut notificator, &logging); @@ -363,12 +363,12 @@ impl Operator for StreamCore { }) } - fn unary(&self, pact: P, name: &str, constructor: B) -> StreamCore + fn unary(self, pact: P, name: &str, constructor: B) -> StreamCore where CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P: ParallelizationContract { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); @@ -391,14 +391,14 @@ impl Operator for StreamCore { stream } - fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore + fn binary_frontier(self, other: StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container + 'static, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut FrontieredInputHandleCore, &mut FrontieredInputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P1: ParallelizationContract, P2: ParallelizationContract { @@ -424,23 +424,23 @@ impl Operator for StreamCore { stream } - fn binary_notify, &mut InputHandleCore, - &mut OutputHandleCore>, + &mut OutputHandleCore>, &mut Notificator)+'static, P1: ParallelizationContract, P2: ParallelizationContract> - (&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { + (self, other: StreamCore, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { + let logging = self.scope().logging(); self.binary_frontier(other, pact1, pact2, name, |capability, _info| { let mut notificator = FrontierNotificator::default(); for time in init { notificator.notify_at(capability.delayed(&time)); } - let logging = self.scope().logging(); move |input1, input2, output| { let frontiers = &[input1.frontier(), input2.frontier()]; let notificator = &mut Notificator::new(frontiers, &mut notificator, &logging); @@ -451,14 +451,14 @@ impl Operator for StreamCore { } - fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore + fn binary(self, other: StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - C2: Container + Data, + C2: Container + 'static, CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, L: FnMut(&mut InputHandleCore, &mut InputHandleCore, - &mut OutputHandleCore>)+'static, + &mut OutputHandleCore>)+'static, P1: ParallelizationContract, P2: ParallelizationContract { @@ -483,7 +483,7 @@ impl Operator for StreamCore { stream } - fn sink(&self, pact: P, name: &str, mut logic: L) + fn sink(self, pact: P, name: &str, mut logic: L) where L: FnMut(&mut FrontieredInputHandleCore)+'static, P: ParallelizationContract { @@ -546,7 +546,7 @@ pub fn source(scope: &G, name: &str, constructor: B) -> Stre where CB: ContainerBuilder, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut OutputHandleCore>)+'static { + L: FnMut(&mut OutputHandleCore>)+'static { let mut builder = OperatorBuilder::new(name.to_owned(), scope.clone()); let operator_info = builder.operator_info(); @@ -586,7 +586,7 @@ where /// /// }); /// ``` -pub fn empty(scope: &G) -> StreamCore { +pub fn empty(scope: &G) -> StreamCore { source::<_, CapacityContainerBuilder, _, _>(scope, "Empty", |_capability, _info| |_output| { // drop capability, do nothing }) diff --git a/timely/src/dataflow/operators/input.rs b/timely/src/dataflow/operators/input.rs index 30d243ab4..ac1afc045 100644 --- a/timely/src/dataflow/operators/input.rs +++ b/timely/src/dataflow/operators/input.rs @@ -1,6 +1,5 @@ //! Create new `Streams` connected to external inputs. -use crate::Data; use crate::container::CapacityContainerBuilder; use crate::dataflow::{Stream, ScopeParent, Scope}; use crate::dataflow::operators::core::{Input as InputCore}; @@ -47,7 +46,7 @@ pub trait Input : Scope { /// } /// }); /// ``` - fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream); + fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream); /// Create a new stream from a supplied interactive handle. /// @@ -79,16 +78,16 @@ pub trait Input : Scope { /// } /// }); /// ``` - fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream; + fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream; } use crate::order::TotalOrder; impl Input for G where ::Timestamp: TotalOrder { - fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream) { + fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream) { InputCore::new_input(self) } - fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream { + fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream { InputCore::input_from(self, handle) } } diff --git a/timely/src/dataflow/operators/map.rs b/timely/src/dataflow/operators/map.rs index 5adab6e10..425b9f208 100644 --- a/timely/src/dataflow/operators/map.rs +++ b/timely/src/dataflow/operators/map.rs @@ -1,13 +1,12 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::Data; use crate::dataflow::{Stream, Scope}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::operator::Operator; use crate::dataflow::operators::core::{Map as MapCore}; /// Extension trait for `Stream`. -pub trait Map { +pub trait Map : Sized { /// Consumes each element of the stream and yields a new element. /// /// # Examples @@ -20,7 +19,7 @@ pub trait Map { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn mapD2+'static>(&self, mut logic: L) -> Stream { + fn mapD2+'static>(self, mut logic: L) -> Stream { self.flat_map(move |x| std::iter::once(logic(x))) } /// Updates each element of the stream and yields the element, re-using memory where possible. @@ -35,7 +34,7 @@ pub trait Map { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn map_in_place(&self, logic: L) -> Stream; + fn map_in_place(self, logic: L) -> Stream; /// Consumes each element of the stream and yields some number of new elements. /// /// # Examples @@ -48,11 +47,11 @@ pub trait Map { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn flat_mapI+'static>(&self, logic: L) -> Stream where I::Item: Data; + fn flat_mapI+'static>(self, logic: L) -> Stream where I::Item: 'static; } -impl Map for Stream { - fn map_in_place(&self, mut logic: L) -> Stream { +impl Map for Stream { + fn map_in_place(self, mut logic: L) -> Stream { self.unary(Pipeline, "MapInPlace", move |_,_| move |input, output| { input.for_each(|time, data| { for datum in data.iter_mut() { logic(datum); } @@ -63,7 +62,7 @@ impl Map for Stream { // 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_mapI+'static>(&self, logic: L) -> Stream where I::Item: Data { + fn flat_mapI+'static>(self, logic: L) -> Stream where I::Item: 'static { MapCore::flat_map(self, logic) } } diff --git a/timely/src/dataflow/operators/partition.rs b/timely/src/dataflow/operators/partition.rs index 39fc427cc..107767592 100644 --- a/timely/src/dataflow/operators/partition.rs +++ b/timely/src/dataflow/operators/partition.rs @@ -3,10 +3,9 @@ use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::{Scope, Stream}; -use crate::Data; /// Partition a stream of records into multiple streams. -pub trait Partition (u64, D2)> { +pub trait Partition (u64, D2)> { /// Produces `parts` output streams, containing records produced and assigned by `route`. /// /// # Examples @@ -16,17 +15,17 @@ pub trait Partition (u64, D2)> { /// timely::example(|scope| { /// let streams = (0..10).to_stream(scope) /// .partition(3, |x| (x % 3, x)); - /// - /// streams[0].inspect(|x| println!("seen 0: {:?}", x)); - /// streams[1].inspect(|x| println!("seen 1: {:?}", x)); - /// streams[2].inspect(|x| println!("seen 2: {:?}", x)); + /// for (index, stream) in streams.into_iter().enumerate() { + /// assert!(index < 3); + /// stream.inspect(move |x| println!("seen {index}: {:?}", x)); + /// } /// }); /// ``` - fn partition(&self, parts: u64, route: F) -> Vec>; + fn partition(self, parts: u64, route: F) -> Vec>; } -impl(u64, D2)+'static> Partition for Stream { - fn partition(&self, parts: u64, route: F) -> Vec> { +impl(u64, D2)+'static> Partition for Stream { + fn partition(self, parts: u64, route: F) -> Vec> { let mut builder = OperatorBuilder::new("Partition".to_owned(), self.scope()); let mut input = builder.new_input(self, Pipeline); diff --git a/timely/src/dataflow/operators/result.rs b/timely/src/dataflow/operators/result.rs index 6dddf8a71..b57124956 100644 --- a/timely/src/dataflow/operators/result.rs +++ b/timely/src/dataflow/operators/result.rs @@ -1,11 +1,10 @@ //! Extension methods for `Stream` containing `Result`s. -use crate::Data; use crate::dataflow::operators::Map; use crate::dataflow::{Scope, Stream}; /// Extension trait for `Stream`. -pub trait ResultStream { +pub trait ResultStream { /// Returns a new instance of `self` containing only `ok` records. /// /// # Examples @@ -18,7 +17,7 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn ok(&self) -> Stream; + fn ok(self) -> Stream; /// Returns a new instance of `self` containing only `err` records. /// @@ -32,7 +31,7 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn err(&self) -> Stream; + fn err(self) -> Stream; /// Returns a new instance of `self` applying `logic` on all `Ok` records. /// @@ -46,7 +45,7 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn map_ok T2 + 'static>(&self, logic: L) -> Stream>; + fn map_ok T2 + 'static>(self, logic: L) -> Stream>; /// Returns a new instance of `self` applying `logic` on all `Err` records. /// @@ -60,7 +59,7 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn map_err E2 + 'static>(&self, logic: L) -> Stream>; + fn map_err E2 + 'static>(self, logic: L) -> Stream>; /// Returns a new instance of `self` applying `logic` on all `Ok` records, passes through `Err` /// records. @@ -75,8 +74,8 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn and_then Result + 'static>( - &self, + fn and_then Result + 'static>( + self, logic: L, ) -> Stream>; @@ -92,31 +91,31 @@ pub trait ResultStream { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn unwrap_or_else T + 'static>(&self, logic: L) -> Stream; + fn unwrap_or_else T + 'static>(self, logic: L) -> Stream; } -impl ResultStream for Stream> { - fn ok(&self) -> Stream { +impl ResultStream for Stream> { + fn ok(self) -> Stream { self.flat_map(Result::ok) } - fn err(&self) -> Stream { + fn err(self) -> Stream { self.flat_map(Result::err) } - fn map_ok T2 + 'static>(&self, mut logic: L) -> Stream> { + fn map_ok T2 + 'static>(self, mut logic: L) -> Stream> { self.map(move |r| r.map(&mut logic)) } - fn map_err E2 + 'static>(&self, mut logic: L) -> Stream> { + fn map_err E2 + 'static>(self, mut logic: L) -> Stream> { self.map(move |r| r.map_err(&mut logic)) } - fn and_then Result + 'static>(&self, mut logic: L) -> Stream> { + fn and_then Result + 'static>(self, mut logic: L) -> Stream> { self.map(move |r| r.and_then(&mut logic)) } - fn unwrap_or_else T + 'static>(&self, mut logic: L) -> Stream { + fn unwrap_or_else T + 'static>(self, mut logic: L) -> Stream { self.map(move |r| r.unwrap_or_else(&mut logic)) } } diff --git a/timely/src/dataflow/operators/to_stream.rs b/timely/src/dataflow/operators/to_stream.rs index a85677dc0..b7916fb1c 100644 --- a/timely/src/dataflow/operators/to_stream.rs +++ b/timely/src/dataflow/operators/to_stream.rs @@ -1,11 +1,10 @@ //! Conversion to the `Stream` type from iterators. -use crate::Data; use crate::dataflow::{Stream, Scope}; use crate::dataflow::operators::core::{ToStream as ToStreamCore}; /// Converts to a timely `Stream`. -pub trait ToStream { +pub trait ToStream { /// Converts to a timely `Stream`. /// /// # Examples @@ -25,7 +24,7 @@ pub trait ToStream { fn to_stream(self, scope: &mut S) -> Stream; } -impl ToStream for I where I::Item: Data { +impl ToStream for I { fn to_stream(self, scope: &mut S) -> Stream { ToStreamCore::to_stream(self, scope) } diff --git a/timely/src/dataflow/operators/unordered_input.rs b/timely/src/dataflow/operators/unordered_input.rs index 76fb4329d..7a4f81996 100644 --- a/timely/src/dataflow/operators/unordered_input.rs +++ b/timely/src/dataflow/operators/unordered_input.rs @@ -1,7 +1,5 @@ //! Create new `Streams` connected to external inputs. -use crate::Data; - use crate::container::CapacityContainerBuilder; use crate::dataflow::operators::{ActivateCapability}; use crate::dataflow::operators::core::{UnorderedInput as UnorderedInputCore, UnorderedHandle as UnorderedHandleCore}; @@ -62,12 +60,12 @@ pub trait UnorderedInput { /// assert_eq!(extract[i], (i, vec![i])); /// } /// ``` - fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), Stream); + fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), Stream); } impl UnorderedInput for G { - fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), Stream) { + fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), Stream) { UnorderedInputCore::new_unordered_input(self) } } diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index d071cac45..8118ece25 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -4,22 +4,21 @@ //! operator output. Extension methods on the `Stream` type provide the appearance of higher-level //! declarative programming, while constructing a dataflow graph underneath. -use crate::progress::{Source, Target}; +use std::fmt::{self, Debug}; +use crate::Container; use crate::communication::Push; 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; +use crate::dataflow::channels::pushers::tee::TeeHelper; +use crate::dataflow::channels::pushers::{PushOwned, Tee}; +use crate::progress::{Source, Target}; -/// Abstraction of a stream of `C: Container` records timestamped with `S::Timestamp`. +/// A tee attached to a stream. /// -/// Internally `Stream` maintains a list of data recipients who should be presented with data +/// Internally `Tee` maintains a list of data recipients who should be presented with data /// produced by the source of the stream. -pub struct StreamCore { +pub struct StreamTee { /// The progress identifier of the stream's data source. name: Source, /// The `Scope` containing the stream. @@ -28,7 +27,7 @@ pub struct StreamCore { ports: TeeHelper, } -impl Clone for StreamCore { +impl Clone for StreamTee { fn clone(&self) -> Self { Self { name: self.name.clone(), @@ -44,16 +43,29 @@ impl Clone for StreamCore { } } +/// Abstraction of a stream of `C: Container` records timestamped with `S::Timestamp`. +/// +/// Internally `Stream` has a single target it can push data at. To fan out to multiple +/// targets, the stream can be forked using [`StreamCore::tee`]. +pub struct StreamCore { + /// The progress identifier of the stream's data source. + name: Source, + /// The `Scope` containing the stream. + scope: S, + /// The single pusher interested in the stream's output, if any. + port: PushOwned, +} + /// 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 /// records should actually be sent. The identifier is unique to the edge and is used only for logging purposes. - pub fn connect_to>+'static>(&self, target: Target, pusher: P, identifier: usize) { - + pub fn connect_to>+'static>(self, target: Target, pusher: P, identifier: usize) { let mut logging = self.scope().logging(); logging.as_mut().map(|l| l.log(crate::logging::ChannelsEvent { id: identifier, @@ -63,11 +75,11 @@ impl StreamCore { })); self.scope.add_edge(self.name, target); - self.ports.add_pusher(pusher); + self.port.set_pusher(pusher); } - /// Allocates a `Stream` from a supplied `Source` name and rendezvous point. - pub fn new(source: Source, output: TeeHelper, scope: S) -> Self { - Self { name: source, ports: output, scope } + /// Allocates a `Stream` from a supplied `Source` name and rendezvous point within a scope. + pub fn new(source: Source, output: PushOwned, scope: S) -> Self { + Self { name: source, port: output, scope } } /// The name of the stream's source operator. pub fn name(&self) -> &Source { &self.name } @@ -76,6 +88,32 @@ impl StreamCore { /// Allows the assertion of a container type, for the benefit of type inference. pub fn container(self) -> StreamCore where Self: AsStream { self.as_stream() } + + /// Convert the stream into a `Tee` that can be cloned. Requires elements to be `Clone`. + /// Consumes this stream. + pub fn tee(self) -> StreamTee where C: Clone { + let (target, registrar) = Tee::new(); + self.port.set_pusher(target); + StreamTee::new(self.name, registrar, self.scope) + } +} + +impl StreamTee { + /// Allocates a `Tee` from a supplied `Source` name and rendezvous point. + pub fn new(source: Source, output: TeeHelper, scope: S) -> Self { + Self { name: source, ports: output, scope } + } + /// The name of the stream's source operator. + pub fn name(&self) -> &Source { &self.name } + /// The scope immediately containing the stream. + pub fn scope(&self) -> S { self.scope.clone() } + + /// Create a `Stream` that attaches to the tee. + pub fn owned(&self) -> StreamCore { + let (target, registrar) = PushOwned::new(); + self.ports.add_pusher(target); + StreamCore::new(self.name, registrar, self.scope()) + } } /// A type that can be translated to a [StreamCore]. @@ -99,3 +137,27 @@ where .finish() } } + +#[cfg(test)] +mod tests { + use crate::dataflow::channels::pact::Pipeline; + use crate::dataflow::operators::{Operator, ToStream}; + + #[derive(Debug, Eq, PartialEq)] + struct NotClone; + + #[test] + fn test_non_clone_stream() { + crate::example(|scope| { + let _ = [NotClone] + .to_stream(scope) + .sink(Pipeline, "check non-clone", |input| { + while let Some((_time, data)) = input.next() { + for datum in data.drain(..) { + assert_eq!(datum, NotClone); + } + } + }); + }); + } +} diff --git a/timely/src/lib.rs b/timely/src/lib.rs index 8be417d47..419276b37 100644 --- a/timely/src/lib.rs +++ b/timely/src/lib.rs @@ -98,18 +98,12 @@ pub mod logging; pub mod scheduling; -/// A composite trait for types usable as data in timely dataflow. -/// -/// The `Data` trait is necessary for all types that go along timely dataflow channels. -pub trait Data: Clone+'static { } -impl Data for T { } - /// A composite trait for types usable on exchange channels in timely dataflow. /// /// The `ExchangeData` trait extends `Data` with any requirements imposed by the `timely_communication` /// `Data` trait, which describes requirements for communication along channels. -pub trait ExchangeData: Data + encoding::Data { } -impl ExchangeData for T { } +pub trait ExchangeData: encoding::Data { } +impl ExchangeData for T { } #[doc = include_str!("../../README.md")] #[cfg(doctest)] diff --git a/timely/src/logging.rs b/timely/src/logging.rs index 6043f2d16..0164fff1b 100644 --- a/timely/src/logging.rs +++ b/timely/src/logging.rs @@ -101,7 +101,7 @@ pub trait ProgressEventTimestamp: std::fmt::Debug + std::any::Any { /// `"std::option::Option"`. fn type_name(&self) -> &'static str; } -impl ProgressEventTimestamp for T { +impl ProgressEventTimestamp for T { fn as_any(&self) -> &dyn std::any::Any { self } fn type_name(&self) -> &'static str { std::any::type_name::() } diff --git a/timely/src/synchronization/sequence.rs b/timely/src/synchronization/sequence.rs index 37c61c3f9..883e34d9c 100644 --- a/timely/src/synchronization/sequence.rs +++ b/timely/src/synchronization/sequence.rs @@ -52,7 +52,7 @@ pub struct Sequencer { recv: Rc>>, // sequenced items. } -impl Sequencer { +impl Sequencer { /// Creates a new Sequencer. /// @@ -230,4 +230,4 @@ impl Drop for Sequencer { .expect("Sequencer.activator unavailable") .activate() } -} \ No newline at end of file +}