diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 000000000..77723d750 --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,8 @@ +version: 2 +updates: + - package-ecosystem: "github-actions" + directory: "/" + # Check for updates every Monday + schedule: + interval: "weekly" + diff --git a/.github/workflows/deploy.yml b/.github/workflows/deploy.yml index fa43572fe..7a6b87b18 100644 --- a/.github/workflows/deploy.yml +++ b/.github/workflows/deploy.yml @@ -7,12 +7,12 @@ on: jobs: deploy: - runs-on: ubuntu-20.04 + runs-on: ubuntu-22.04 steps: - - uses: actions/checkout@v2 - - run: cargo install mdbook --version 0.4.6 + - uses: actions/checkout@v3 + - run: cargo install mdbook --version 0.4.27 - run: cd mdbook && mdbook build - - uses: JamesIves/github-pages-deploy-action@4.0.0 + - uses: JamesIves/github-pages-deploy-action@v4 with: branch: gh-pages folder: mdbook/book diff --git a/.github/workflows/release-plz.yml b/.github/workflows/release-plz.yml new file mode 100644 index 000000000..d70eedee0 --- /dev/null +++ b/.github/workflows/release-plz.yml @@ -0,0 +1,28 @@ +name: Release-plz + +permissions: + pull-requests: write + contents: write + +on: + push: + branches: + - master + pull_request: + +jobs: + release-plz: + name: Release-plz + runs-on: ubuntu-latest + steps: + - name: Checkout repository + uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Install Rust toolchain + uses: actions-rust-lang/setup-rust-toolchain@v1 + - name: Run release-plz + uses: MarcoIeni/release-plz-action@v0.5 + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + CARGO_REGISTRY_TOKEN: ${{ secrets.CARGO_REGISTRY_TOKEN }} diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 4d4c3aee0..9a1a6be95 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -4,10 +4,10 @@ on: [push, pull_request] jobs: test: - runs-on: ubuntu-20.04 + runs-on: ubuntu-22.04 steps: - - uses: actions/checkout@v2 - - run: rustup update 1.50.0 --no-self-update && rustup default 1.50.0 + - uses: actions/checkout@v3 + - run: rustup update 1.64 --no-self-update && rustup default 1.64 - run: cargo build - name: test mdBook # rustdoc doesn't build dependencies, so it needs to run after `cargo build`, diff --git a/CHANGELOG.md b/CHANGELOG.md index c499d5d9d..133dcaf37 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -92,7 +92,7 @@ Beneficial fallouts include our ability to remove `RootTimestamp`, as dataflows - The `Root` type has been renamed `Worker` and is found in the `::worker` module. The methods of the `ScopeParent` trait are now in the `::worker::AsWorker` trait. - The communication `Allocate` trait's main method `allocate` now takes a worker-unique identifier to use for the channel. The allocator may or may not use the information (most often for logging), but they are allowed to be incorrect if one allocates two channels with the same identifier. - A `CapabilityRef` now supports `retain_for(usize)` which indicates a specific output port the capability should be retain for use with. The `retain()` method still exists for now and is equivalent to `retain(0)`. This change also comes with the *inability* to use an arbitrary `Capability` with any output; using a capability bound to the wrong output will result in a run-time error. -- The `unary` and `binary` operators now provide `data` as a `RefOrMut`, which does not implement `DerefMut`. More information on how to port methods can be found [here](https://github.com/frankmcsherry/timely-dataflow/pull/135#issuecomment-418355284). +- The `unary` and `binary` operators now provide `data` as a `RefOrMut`, which does not implement `DerefMut`. More information on how to port methods can be found [here](https://github.com/TimelyDataflow/timely-dataflow/pull/135#issuecomment-418355284). ### Removed diff --git a/Cargo.toml b/Cargo.toml index a7feac1a1..755167b78 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,10 +2,9 @@ members = [ "bytes", "communication", - "kafkaesque", + "container", "logging", "timely", - "experiments" ] [profile.release] diff --git a/communication/src/allocator/counters.rs b/communication/src/allocator/counters.rs index 68d1fa4ab..4bee06117 100644 --- a/communication/src/allocator/counters.rs +++ b/communication/src/allocator/counters.rs @@ -2,23 +2,21 @@ use std::rc::Rc; use std::cell::RefCell; -use std::collections::VecDeque; use crate::{Push, Pull}; -use crate::allocator::Event; /// The push half of an intra-thread channel. pub struct Pusher> { index: usize, // count: usize, - events: Rc>>, + events: Rc>>, pusher: P, phantom: ::std::marker::PhantomData, } impl> Pusher { /// Wraps a pusher with a message counter. - pub fn new(pusher: P, index: usize, events: Rc>>) -> Self { + pub fn new(pusher: P, index: usize, events: Rc>>) -> Self { Pusher { index, // count: 0, @@ -36,7 +34,7 @@ impl> Push for Pusher { // if self.count != 0 { // self.events // .borrow_mut() - // .push_back((self.index, Event::Pushed(self.count))); + // .push_back(self.index); // self.count = 0; // } // } @@ -47,7 +45,7 @@ impl> Push for Pusher { // moving information along. Better, but needs cooperation. self.events .borrow_mut() - .push_back((self.index, Event::Pushed(1))); + .push(self.index); self.pusher.push(element) } @@ -59,7 +57,7 @@ use crossbeam_channel::Sender; pub struct ArcPusher> { index: usize, // count: usize, - events: Sender<(usize, Event)>, + events: Sender, pusher: P, phantom: ::std::marker::PhantomData, buzzer: crate::buzzer::Buzzer, @@ -67,7 +65,7 @@ pub struct ArcPusher> { impl> ArcPusher { /// Wraps a pusher with a message counter. - pub fn new(pusher: P, index: usize, events: Sender<(usize, Event)>, buzzer: crate::buzzer::Buzzer) -> Self { + pub fn new(pusher: P, index: usize, events: Sender, buzzer: crate::buzzer::Buzzer) -> Self { ArcPusher { index, // count: 0, @@ -99,7 +97,7 @@ impl> Push for ArcPusher { // and finally awaken the thread. Other orders are defective when // multiple threads are involved. self.pusher.push(element); - let _ = self.events.send((self.index, Event::Pushed(1))); + let _ = self.events.send(self.index); // TODO : Perhaps this shouldn't be a fatal error (e.g. in shutdown). // .expect("Failed to send message count"); self.buzzer.buzz(); @@ -110,14 +108,14 @@ impl> Push for ArcPusher { pub struct Puller> { index: usize, count: usize, - events: Rc>>, + events: Rc>>, puller: P, phantom: ::std::marker::PhantomData, } impl> Puller { /// Wraps a puller with a message counter. - pub fn new(puller: P, index: usize, events: Rc>>) -> Self { + pub fn new(puller: P, index: usize, events: Rc>>) -> Self { Puller { index, count: 0, @@ -135,7 +133,7 @@ impl> Pull for Puller { if self.count != 0 { self.events .borrow_mut() - .push_back((self.index, Event::Pulled(self.count))); + .push(self.index); self.count = 0; } } diff --git a/communication/src/allocator/generic.rs b/communication/src/allocator/generic.rs index 625de6cd6..49f014072 100644 --- a/communication/src/allocator/generic.rs +++ b/communication/src/allocator/generic.rs @@ -5,11 +5,10 @@ use std::rc::Rc; use std::cell::RefCell; -use std::collections::VecDeque; use crate::allocator::thread::ThreadBuilder; use crate::allocator::process::ProcessBuilder as TypedProcessBuilder; -use crate::allocator::{Allocate, AllocateBuilder, Event, Thread, Process}; +use crate::allocator::{Allocate, AllocateBuilder, Thread, Process}; use crate::allocator::zero_copy::allocator_process::{ProcessBuilder, ProcessAllocator}; use crate::allocator::zero_copy::allocator::{TcpBuilder, TcpAllocator}; @@ -74,7 +73,7 @@ impl Generic { Generic::ZeroCopy(z) => z.release(), } } - fn events(&self) -> &Rc>> { + fn events(&self) -> &Rc>> { match self { Generic::Thread(ref t) => t.events(), Generic::Process(ref p) => p.events(), @@ -93,7 +92,7 @@ impl Allocate for Generic { fn receive(&mut self) { self.receive(); } fn release(&mut self) { self.release(); } - fn events(&self) -> &Rc>> { self.events() } + fn events(&self) -> &Rc>> { self.events() } fn await_events(&self, _duration: Option) { match self { Generic::Thread(t) => t.await_events(_duration), diff --git a/communication/src/allocator/mod.rs b/communication/src/allocator/mod.rs index 4c29b85e1..e5b858f69 100644 --- a/communication/src/allocator/mod.rs +++ b/communication/src/allocator/mod.rs @@ -3,7 +3,6 @@ use std::rc::Rc; use std::cell::RefCell; use std::time::Duration; -use std::collections::VecDeque; pub use self::thread::Thread; pub use self::process::Process; @@ -50,7 +49,7 @@ pub trait Allocate { /// drain these events in order to drive their computation. If they /// fail to do so the event queue may become quite large, and turn /// into a performance problem. - fn events(&self) -> &Rc>>; + fn events(&self) -> &Rc>>; /// Awaits communication events. /// @@ -92,11 +91,3 @@ pub trait Allocate { thread::Thread::new_from(identifier, self.events().clone()) } } - -/// A communication channel event. -pub enum Event { - /// A number of messages pushed into the channel. - Pushed(usize), - /// A number of messages pulled from the channel. - Pulled(usize), -} diff --git a/communication/src/allocator/process.rs b/communication/src/allocator/process.rs index 132bc6de5..07d793684 100644 --- a/communication/src/allocator/process.rs +++ b/communication/src/allocator/process.rs @@ -5,11 +5,11 @@ use std::cell::RefCell; use std::sync::{Arc, Mutex}; use std::any::Any; use std::time::Duration; -use std::collections::{HashMap, VecDeque}; +use std::collections::{HashMap}; use crossbeam_channel::{Sender, Receiver}; use crate::allocator::thread::{ThreadBuilder}; -use crate::allocator::{Allocate, AllocateBuilder, Event, Thread}; +use crate::allocator::{Allocate, AllocateBuilder, Thread}; use crate::{Push, Pull, Message}; use crate::buzzer::Buzzer; @@ -25,8 +25,8 @@ pub struct ProcessBuilder { buzzers_send: Vec>, buzzers_recv: Vec>, - counters_send: Vec>, - counters_recv: Receiver<(usize, Event)>, + counters_send: Vec>, + counters_recv: Receiver, } impl AllocateBuilder for ProcessBuilder { @@ -63,8 +63,8 @@ pub struct Process { // below: `Box` is a `Box>, Receiver)>>>` channels: Arc>>>, buzzers: Vec, - counters_send: Vec>, - counters_recv: Receiver<(usize, Event)>, + counters_send: Vec>, + counters_recv: Receiver, } impl Process { @@ -174,7 +174,7 @@ impl Allocate for Process { (sends, recv) } - fn events(&self) -> &Rc>> { + fn events(&self) -> &Rc>> { self.inner.events() } @@ -184,8 +184,8 @@ impl Allocate for Process { fn receive(&mut self) { let mut events = self.inner.events().borrow_mut(); - while let Ok((index, event)) = self.counters_recv.try_recv() { - events.push_back((index, event)); + while let Ok(index) = self.counters_recv.try_recv() { + events.push(index); } } } diff --git a/communication/src/allocator/thread.rs b/communication/src/allocator/thread.rs index ba5407e4d..f46e3532b 100644 --- a/communication/src/allocator/thread.rs +++ b/communication/src/allocator/thread.rs @@ -5,7 +5,7 @@ use std::cell::RefCell; use std::time::Duration; use std::collections::VecDeque; -use crate::allocator::{Allocate, AllocateBuilder, Event}; +use crate::allocator::{Allocate, AllocateBuilder}; use crate::allocator::counters::Pusher as CountPusher; use crate::allocator::counters::Puller as CountPuller; use crate::{Push, Pull, Message}; @@ -22,7 +22,7 @@ impl AllocateBuilder for ThreadBuilder { /// An allocator for intra-thread communication. pub struct Thread { /// Shared counts of messages in channels. - events: Rc>>, + events: Rc>>, } impl Allocate for Thread { @@ -32,7 +32,7 @@ impl Allocate for Thread { let (pusher, puller) = Thread::new_from(identifier, self.events.clone()); (vec![Box::new(pusher)], Box::new(puller)) } - fn events(&self) -> &Rc>> { + fn events(&self) -> &Rc>> { &self.events } fn await_events(&self, duration: Option) { @@ -56,12 +56,12 @@ impl Thread { /// Allocates a new thread-local channel allocator. pub fn new() -> Self { Thread { - events: Rc::new(RefCell::new(VecDeque::new())), + events: Rc::new(RefCell::new(Default::default())), } } /// Creates a new thread-local channel from an identifier and shared counts. - pub fn new_from(identifier: usize, events: Rc>>) + pub fn new_from(identifier: usize, events: Rc>>) -> (ThreadPusher>, ThreadPuller>) { let shared = Rc::new(RefCell::new((VecDeque::>::new(), VecDeque::>::new()))); diff --git a/communication/src/allocator/zero_copy/allocator.rs b/communication/src/allocator/zero_copy/allocator.rs index 64225ff29..6ef9ef647 100644 --- a/communication/src/allocator/zero_copy/allocator.rs +++ b/communication/src/allocator/zero_copy/allocator.rs @@ -10,7 +10,6 @@ use crate::networking::MessageHeader; use crate::{Allocate, Message, Data, Push, Pull}; use crate::allocator::AllocateBuilder; -use crate::allocator::Event; use crate::allocator::canary::Canary; use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue}; @@ -229,7 +228,7 @@ impl Allocate for TcpAllocator { // Increment message count for channel. // Safe to do this even if the channel has been dropped. - events.push_back((header.channel, Event::Pushed(1))); + events.push(header.channel); // Ensure that a queue exists. match self.to_local.entry(header.channel) { @@ -269,7 +268,7 @@ impl Allocate for TcpAllocator { // } // } } - fn events(&self) -> &Rc>> { + fn events(&self) -> &Rc>> { self.inner.events() } fn await_events(&self, duration: Option) { diff --git a/communication/src/allocator/zero_copy/allocator_process.rs b/communication/src/allocator/zero_copy/allocator_process.rs index dd2815a50..74056ac29 100644 --- a/communication/src/allocator/zero_copy/allocator_process.rs +++ b/communication/src/allocator/zero_copy/allocator_process.rs @@ -10,7 +10,7 @@ use bytes::arc::Bytes; use crate::networking::MessageHeader; use crate::{Allocate, Message, Data, Push, Pull}; -use crate::allocator::{AllocateBuilder, Event}; +use crate::allocator::{AllocateBuilder}; use crate::allocator::canary::Canary; use super::bytes_exchange::{BytesPull, SendEndpoint, MergeQueue}; @@ -77,7 +77,7 @@ impl ProcessBuilder { ProcessAllocator { index: self.index, peers: self.peers, - events: Rc::new(RefCell::new(VecDeque::new())), + events: Rc::new(RefCell::new(Default::default())), canaries: Rc::new(RefCell::new(Vec::new())), channel_id_bound: None, staged: Vec::new(), @@ -103,7 +103,7 @@ pub struct ProcessAllocator { index: usize, // number out of peers peers: usize, // number of peer allocators (for typed channel allocation). - events: Rc>>, + events: Rc>>, canaries: Rc>>, @@ -196,7 +196,7 @@ impl Allocate for ProcessAllocator { // Increment message count for channel. // Safe to do this even if the channel has been dropped. - events.push_back((header.channel, Event::Pushed(1))); + events.push(header.channel); // Ensure that a queue exists. match self.to_local.entry(header.channel) { @@ -237,7 +237,7 @@ impl Allocate for ProcessAllocator { // } } - fn events(&self) -> &Rc>> { + fn events(&self) -> &Rc>> { &self.events } fn await_events(&self, duration: Option) { diff --git a/communication/src/allocator/zero_copy/initialize.rs b/communication/src/allocator/zero_copy/initialize.rs index 97ade2c78..be1494fa4 100644 --- a/communication/src/allocator/zero_copy/initialize.rs +++ b/communication/src/allocator/zero_copy/initialize.rs @@ -6,6 +6,7 @@ use crate::allocator::process::ProcessBuilder; use crate::networking::create_sockets; use super::tcp::{send_loop, recv_loop}; use super::allocator::{TcpBuilder, new_vector}; +use super::stream::Stream; /// Join handles for send and receive threads. /// @@ -52,8 +53,8 @@ pub fn initialize_networking( /// /// It is important that the `sockets` argument contain sockets for each remote process, in order, and /// with position `my_index` set to `None`. -pub fn initialize_networking_from_sockets( - mut sockets: Vec>, +pub fn initialize_networking_from_sockets( + mut sockets: Vec>, my_index: usize, threads: usize, log_sender: BoxOption>+Send+Sync>) diff --git a/communication/src/allocator/zero_copy/mod.rs b/communication/src/allocator/zero_copy/mod.rs index e4bf9e396..81b90c01f 100644 --- a/communication/src/allocator/zero_copy/mod.rs +++ b/communication/src/allocator/zero_copy/mod.rs @@ -10,4 +10,5 @@ pub mod tcp; pub mod allocator; pub mod allocator_process; pub mod initialize; -pub mod push_pull; \ No newline at end of file +pub mod push_pull; +pub mod stream; \ No newline at end of file diff --git a/communication/src/allocator/zero_copy/stream.rs b/communication/src/allocator/zero_copy/stream.rs new file mode 100644 index 000000000..854648a08 --- /dev/null +++ b/communication/src/allocator/zero_copy/stream.rs @@ -0,0 +1,47 @@ +//! Abstractions over network streams. + +use std::io; +use std::net::{TcpStream, Shutdown}; +#[cfg(unix)] +use std::os::unix::net::UnixStream; + +/// An abstraction over network streams. +pub trait Stream: Sized + Send + Sync + io::Read + io::Write { + /// Creates a new independently owned handle to the underlying stream. + fn try_clone(&self) -> io::Result; + + /// Moves this stream into or out of nonblocking mode. + fn set_nonblocking(&self, nonblocking: bool) -> io::Result<()>; + + /// Shuts down the read, write, or both halves of this connection. + fn shutdown(&self, how: Shutdown) -> io::Result<()>; +} + +impl Stream for TcpStream { + fn try_clone(&self) -> io::Result { + self.try_clone() + } + + fn set_nonblocking(&self, nonblocking: bool) -> io::Result<()> { + self.set_nonblocking(nonblocking) + } + + fn shutdown(&self, how: Shutdown) -> io::Result<()> { + self.shutdown(how) + } +} + +#[cfg(unix)] +impl Stream for UnixStream { + fn try_clone(&self) -> io::Result { + self.try_clone() + } + + fn set_nonblocking(&self, nonblocking: bool) -> io::Result<()> { + self.set_nonblocking(nonblocking) + } + + fn shutdown(&self, how: Shutdown) -> io::Result<()> { + self.shutdown(how) + } +} diff --git a/communication/src/allocator/zero_copy/tcp.rs b/communication/src/allocator/zero_copy/tcp.rs index 5f3855b87..f2099f516 100644 --- a/communication/src/allocator/zero_copy/tcp.rs +++ b/communication/src/allocator/zero_copy/tcp.rs @@ -1,31 +1,43 @@ //! -use std::io::{Read, Write}; -use std::net::TcpStream; +use std::io::{self, Write}; use crossbeam_channel::{Sender, Receiver}; use crate::networking::MessageHeader; use super::bytes_slab::BytesSlab; use super::bytes_exchange::MergeQueue; +use super::stream::Stream; use logging_core::Logger; use crate::logging::{CommunicationEvent, CommunicationSetup, MessageEvent, StateEvent}; +fn tcp_panic(context: &'static str, cause: io::Error) -> ! { + // NOTE: some downstream crates sniff out "timely communication error:" from + // the panic message. Avoid removing or rewording this message if possible. + // It'd be nice to instead use `panic_any` here with a structured error + // type, but the panic message for `panic_any` is no good (Box). + panic!("timely communication error: {}: {}", context, cause) +} + /// Repeatedly reads from a TcpStream and carves out messages. /// /// The intended communication pattern is a sequence of (header, message)^* for valid /// messages, followed by a header for a zero length message indicating the end of stream. -/// If the stream ends without being shut down, the receive thread panics in an attempt to -/// take down the computation and cause the failures to cascade. -pub fn recv_loop( - mut reader: TcpStream, +/// +/// If the stream ends without being shut down, or if reading from the stream fails, the +/// receive thread panics with a message that starts with "timely communication error:" +/// in an attempt to take down the computation and cause the failures to cascade. +pub fn recv_loop( + mut reader: S, targets: Vec>, worker_offset: usize, process: usize, remote: usize, mut logger: Option>) +where + S: Stream, { // Log the receive thread's start. logger.as_mut().map(|l| l.log(StateEvent { send: false, process, remote, start: true })); @@ -56,15 +68,16 @@ pub fn recv_loop( // Attempt to read some more bytes into self.buffer. let read = match reader.read(&mut buffer.empty()) { + Err(x) => tcp_panic("reading data", x), + Ok(n) if n == 0 => { + tcp_panic( + "reading data", + std::io::Error::new(std::io::ErrorKind::UnexpectedEof, "socket closed"), + ); + } Ok(n) => n, - Err(x) => { - // We don't expect this, as socket closure results in Ok(0) reads. - println!("Error: {:?}", x); - 0 - }, }; - assert!(read > 0); buffer.make_valid(read); // Consume complete messages from the front of self.buffer. @@ -89,7 +102,7 @@ pub fn recv_loop( panic!("Clean shutdown followed by data."); } buffer.ensure_capacity(1); - if reader.read(&mut buffer.empty()).expect("read failure") > 0 { + if reader.read(&mut buffer.empty()).unwrap_or_else(|e| tcp_panic("reading EOF", e)) > 0 { panic!("Clean shutdown followed by data."); } } @@ -111,9 +124,13 @@ pub fn recv_loop( /// /// The intended communication pattern is a sequence of (header, message)^* for valid /// messages, followed by a header for a zero length message indicating the end of stream. -pub fn send_loop( +/// +/// If writing to the stream fails, the send thread panics with a message that starts with +/// "timely communication error:" in an attempt to take down the computation and cause the +/// failures to cascade. +pub fn send_loop( // TODO: Maybe we don't need BufWriter with consolidation in writes. - writer: TcpStream, + writer: S, sources: Vec>, process: usize, remote: usize, @@ -148,7 +165,7 @@ pub fn send_loop( // still be a signal incoming. // // We could get awoken by more data, a channel closing, or spuriously perhaps. - writer.flush().expect("Failed to flush writer."); + writer.flush().unwrap_or_else(|e| tcp_panic("flushing writer", e)); sources.retain(|source| !source.is_complete()); if !sources.is_empty() { std::thread::park(); @@ -167,7 +184,7 @@ pub fn send_loop( } }); - writer.write_all(&bytes[..]).expect("Write failure in send_loop."); + writer.write_all(&bytes[..]).unwrap_or_else(|e| tcp_panic("writing data", e)); } } } @@ -182,9 +199,9 @@ pub fn send_loop( length: 0, seqno: 0, }; - header.write_to(&mut writer).expect("Failed to write header!"); - writer.flush().expect("Failed to flush writer."); - writer.get_mut().shutdown(::std::net::Shutdown::Write).expect("Write shutdown failed"); + header.write_to(&mut writer).unwrap_or_else(|e| tcp_panic("writing data", e)); + writer.flush().unwrap_or_else(|e| tcp_panic("flushing writer", e)); + writer.get_mut().shutdown(::std::net::Shutdown::Write).unwrap_or_else(|e| tcp_panic("shutting down writer", e)); logger.as_mut().map(|logger| logger.log(MessageEvent { is_send: true, header })); // Log the send thread's end. diff --git a/communication/src/lib.rs b/communication/src/lib.rs index 66ed3e0a3..e3302cdb0 100644 --- a/communication/src/lib.rs +++ b/communication/src/lib.rs @@ -4,7 +4,7 @@ //! It may be independently useful, but it is separated out mostly to make clear boundaries in the project. //! //! Threads are spawned with an [`allocator::Generic`](allocator::generic::Generic), whose -//! [`allocate`](allocator::generic::Generic::allocate) method returns a pair of several send endpoints and one +//! [`allocate`](Allocate::allocate) method returns a pair of several send endpoints and one //! receive endpoint. Messages sent into a send endpoint will eventually be received by the corresponding worker, //! if it receives often enough. The point-to-point channels are each FIFO, but with no fairness guarantees. //! diff --git a/communication/src/message.rs b/communication/src/message.rs index b706c5c15..1289498e8 100644 --- a/communication/src/message.rs +++ b/communication/src/message.rs @@ -49,6 +49,16 @@ impl<'a, T: Clone+'a> RefOrMut<'a, T> { self.swap(&mut element); element } + + /// Extracts the contents of `self`, either by cloning, or swapping and leaving a default + /// element in place. + /// + /// This consumes `self` because its contents are now in an unknown state. + pub fn take(self) -> T where T: Default { + let mut element = Default::default(); + self.swap(&mut element); + element + } } /// A wrapped message which may be either typed or binary data. diff --git a/container/Cargo.toml b/container/Cargo.toml new file mode 100644 index 000000000..6b3e2d65d --- /dev/null +++ b/container/Cargo.toml @@ -0,0 +1,10 @@ +[package] +name = "timely_container" +version = "0.12.0" +edition = "2018" +description = "Container abstractions for Timely" +license = "MIT" + +[dependencies] +columnation = { git = "https://github.com/frankmcsherry/columnation" } +serde = { version = "1.0"} diff --git a/container/src/columnation.rs b/container/src/columnation.rs new file mode 100644 index 000000000..54a00ca4e --- /dev/null +++ b/container/src/columnation.rs @@ -0,0 +1,343 @@ +//! A columnar container based on the columnation library. + +use std::iter::FromIterator; + +pub use columnation::*; + +/// An append-only vector that store records as columns. +/// +/// This container maintains elements that might conventionally own +/// memory allocations, but instead the pointers to those allocations +/// reference larger regions of memory shared with multiple instances +/// of the type. Elements can be retrieved as references, and care is +/// taken when this type is dropped to ensure that the correct memory +/// is returned (rather than the incorrect memory, from running the +/// elements `Drop` implementations). +pub struct TimelyStack { + local: Vec, + inner: T::InnerRegion, +} + +impl TimelyStack { + /// Construct a [TimelyStack], reserving space for `capacity` elements + /// + /// Note that the associated region is not initialized to a specific capacity + /// because we can't generally know how much space would be required. + pub fn with_capacity(capacity: usize) -> Self { + Self { + local: Vec::with_capacity(capacity), + inner: T::InnerRegion::default(), + } + } + + /// Ensures `Self` can absorb `items` without further allocations. + /// + /// The argument `items` may be cloned and iterated multiple times. + /// Please be careful if it contains side effects. + #[inline(always)] + pub fn reserve_items<'a, I>(&'a mut self, items: I) + where + I: Iterator+Clone, + { + self.local.reserve(items.clone().count()); + self.inner.reserve_items(items); + } + + /// Ensures `Self` can absorb `items` without further allocations. + /// + /// The argument `items` may be cloned and iterated multiple times. + /// Please be careful if it contains side effects. + #[inline(always)] + pub fn reserve_regions<'a, I>(&mut self, regions: I) + where + Self: 'a, + I: Iterator+Clone, + { + self.local.reserve(regions.clone().map(|cs| cs.local.len()).sum()); + self.inner.reserve_regions(regions.map(|cs| &cs.inner)); + } + + + + /// Copies an element in to the region. + /// + /// The element can be read by indexing + pub fn copy(&mut self, item: &T) { + // TODO: Some types `T` should just be cloned. + // E.g. types that are `Copy` or vecs of ZSTs. + unsafe { + self.local.push(self.inner.copy(item)); + } + } + /// Empties the collection. + pub fn clear(&mut self) { + unsafe { + // Unsafety justified in that setting the length to zero exposes + // no invalid data. + self.local.set_len(0); + self.inner.clear(); + } + } + /// Retain elements that pass a predicate, from a specified offset. + /// + /// This method may or may not reclaim memory in the inner region. + pub fn retain_from bool>(&mut self, index: usize, mut predicate: P) { + let mut write_position = index; + for position in index..self.local.len() { + if predicate(&self[position]) { + // TODO: compact the inner region and update pointers. + self.local.swap(position, write_position); + write_position += 1; + } + } + unsafe { + // Unsafety justified in that `write_position` is no greater than + // `self.local.len()` and so this exposes no invalid data. + self.local.set_len(write_position); + } + } + + /// Unsafe access to `local` data. The slices stores data that is backed by a region + /// allocation. Therefore, it is undefined behavior to mutate elements of the `local` slice. + /// + /// # Safety + /// Elements within `local` can be reordered, but not mutated, removed and/or dropped. + pub unsafe fn local(&mut self) -> &mut [T] { + &mut self.local[..] + } + + /// Estimate the memory capacity in bytes. + #[inline] + pub fn heap_size(&self, mut callback: impl FnMut(usize, usize)) { + let size_of = std::mem::size_of::(); + callback(self.local.len() * size_of, self.local.capacity() * size_of); + self.inner.heap_size(callback); + } + + /// Estimate the consumed memory capacity in bytes, summing both used and total capacity. + #[inline] + pub fn summed_heap_size(&self) -> (usize, usize) { + let (mut length, mut capacity) = (0, 0); + self.heap_size(|len, cap| { + length += len; + capacity += cap + }); + (length, capacity) + } +} + +impl TimelyStack<(A, B)> { + /// Copies a destructured tuple `(A, B)` into this column stack. + /// + /// This serves situations where a tuple should be constructed from its constituents but not + /// not all elements are available as owned data. + /// + /// The element can be read by indexing + pub fn copy_destructured(&mut self, t1: &A, t2: &B) { + unsafe { + self.local.push(self.inner.copy_destructured(t1, t2)); + } + } +} + +impl TimelyStack<(A, B, C)> { + /// Copies a destructured tuple `(A, B, C)` into this column stack. + /// + /// This serves situations where a tuple should be constructed from its constituents but not + /// not all elements are available as owned data. + /// + /// The element can be read by indexing + pub fn copy_destructured(&mut self, r0: &A, r1: &B, r2: &C) { + unsafe { + self.local.push(self.inner.copy_destructured(r0, r1, r2)); + } + } +} + +impl std::ops::Deref for TimelyStack { + type Target = [T]; + #[inline(always)] + fn deref(&self) -> &Self::Target { + &self.local[..] + } +} + +impl Drop for TimelyStack { + fn drop(&mut self) { + self.clear(); + } +} + +impl Default for TimelyStack { + fn default() -> Self { + Self { + local: Vec::new(), + inner: T::InnerRegion::default(), + } + } +} + +impl<'a, A: 'a + Columnation> FromIterator<&'a A> for TimelyStack { + fn from_iter>(iter: T) -> Self { + let mut iter = iter.into_iter(); + let mut c = TimelyStack::::with_capacity(iter.size_hint().0); + while let Some(element) = iter.next() { + c.copy(element); + } + + c + } +} + +impl PartialEq for TimelyStack { + fn eq(&self, other: &Self) -> bool { + PartialEq::eq(&self[..], &other[..]) + } +} + +impl Eq for TimelyStack {} + +impl std::fmt::Debug for TimelyStack { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + (&self[..]).fmt(f) + } +} + +impl Clone for TimelyStack { + fn clone(&self) -> Self { + let mut new: Self = Default::default(); + for item in &self[..] { + new.copy(item); + } + new + } + + fn clone_from(&mut self, source: &Self) { + self.clear(); + for item in &source[..] { + self.copy(item); + } + } +} + +mod serde { + use serde::{Deserialize, Deserializer, Serialize, Serializer}; + + use crate::columnation::{Columnation, TimelyStack}; + + impl Serialize for TimelyStack { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + use serde::ser::SerializeSeq; + let mut seq = serializer.serialize_seq(Some(self.local.len()))?; + for element in &self[..] { + seq.serialize_element(element)?; + } + seq.end() + } + } + + impl<'a, T: Columnation + Deserialize<'a>> Deserialize<'a> for TimelyStack { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'a>, + { + use serde::de::{SeqAccess, Visitor}; + use std::fmt; + use std::marker::PhantomData; + struct TimelyStackVisitor { + marker: PhantomData, + } + + impl<'de, T: Columnation> Visitor<'de> for TimelyStackVisitor + where + T: Deserialize<'de>, + { + type Value = TimelyStack; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("a sequence") + } + + fn visit_seq(self, mut seq: A) -> Result + where + A: SeqAccess<'de>, + { + let local = Vec::with_capacity( + seq.size_hint() + .unwrap_or(crate::buffer::default_capacity::()), + ); + let mut stack = TimelyStack { + local, + inner: T::InnerRegion::default(), + }; + + while let Some(value) = seq.next_element()? { + stack.copy(&value); + } + + Ok(stack) + } + } + + let visitor = TimelyStackVisitor { + marker: PhantomData, + }; + deserializer.deserialize_seq(visitor) + } + } +} + +mod container { + use crate::{Container, PushPartitioned}; + + use crate::columnation::{Columnation, TimelyStack}; + + impl Container for TimelyStack { + type Item = T; + + fn len(&self) -> usize { + self.local.len() + } + + fn is_empty(&self) -> bool { + self.local.is_empty() + } + + fn capacity(&self) -> usize { + self.local.capacity() + } + + fn clear(&mut self) { + TimelyStack::clear(self) + } + } + + impl PushPartitioned for TimelyStack { + fn push_partitioned(&mut self, buffers: &mut [Self], mut index: I, mut flush: F) + where + I: FnMut(&Self::Item) -> usize, + F: FnMut(usize, &mut Self), + { + fn ensure_capacity(this: &mut TimelyStack) { + let capacity = this.local.capacity(); + let desired_capacity = crate::buffer::default_capacity::(); + if capacity < desired_capacity { + this.local.reserve(desired_capacity - capacity); + } + } + + for datum in &self[..] { + let index = index(&datum); + ensure_capacity(&mut buffers[index]); + buffers[index].copy(datum); + if buffers[index].len() == buffers[index].local.capacity() { + flush(index, &mut buffers[index]); + } + } + self.clear(); + } + } +} diff --git a/container/src/lib.rs b/container/src/lib.rs new file mode 100644 index 000000000..eb9b3ce77 --- /dev/null +++ b/container/src/lib.rs @@ -0,0 +1,179 @@ +//! Specifications for containers + +#![forbid(missing_docs)] + +pub mod columnation; + +/// A container transferring data through dataflow edges +/// +/// A container stores a number of elements and thus is able to describe it length (`len()`) and +/// whether it is empty (`is_empty()`). It supports removing all elements (`clear`). +/// +/// 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`.) +/// TODO: Don't require `Container: Clone` +pub trait Container: Default + Clone + 'static { + /// The type of elements this container holds. + type Item; + + /// The number of elements in this container + /// + /// The length of a container must be consistent between sending and receiving it. + /// When exchanging a container and partitioning it into pieces, the sum of the length + /// of all pieces must be equal to the length of the original container. + fn len(&self) -> usize; + + /// Determine if the container contains any elements, corresponding to `len() == 0`. + fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// The capacity of the underlying container + fn capacity(&self) -> usize; + + /// Remove all contents from `self` while retaining allocated memory. + /// After calling `clear`, `is_empty` must return `true` and `len` 0. + fn clear(&mut self); +} + +impl Container for Vec { + type Item = T; + + fn len(&self) -> usize { + Vec::len(self) + } + + fn is_empty(&self) -> bool { + Vec::is_empty(self) + } + + fn capacity(&self) -> usize { + Vec::capacity(self) + } + + fn clear(&mut self) { Vec::clear(self) } +} + +mod rc { + use std::rc::Rc; + + use crate::Container; + + impl Container for Rc { + type Item = T::Item; + + fn len(&self) -> usize { + std::ops::Deref::deref(self).len() + } + + fn is_empty(&self) -> bool { + std::ops::Deref::deref(self).is_empty() + } + + fn capacity(&self) -> usize { + std::ops::Deref::deref(self).capacity() + } + + fn clear(&mut self) { + // Try to reuse the allocation if possible + if let Some(inner) = Rc::get_mut(self) { + inner.clear(); + } else { + *self = Self::default(); + } + } + } +} + +mod arc { + use std::sync::Arc; + + use crate::Container; + + impl Container for Arc { + type Item = T::Item; + + fn len(&self) -> usize { + std::ops::Deref::deref(self).len() + } + + fn is_empty(&self) -> bool { + std::ops::Deref::deref(self).is_empty() + } + + fn capacity(&self) -> usize { + std::ops::Deref::deref(self).capacity() + } + + fn clear(&mut self) { + // Try to reuse the allocation if possible + if let Some(inner) = Arc::get_mut(self) { + inner.clear(); + } else { + *self = Self::default(); + } + } + } +} + +/// A container that can partition itself into pieces. +pub trait PushPartitioned: Container { + /// Partition and push this container. + /// + /// Drain all elements from `self`, and use the function `index` to determine which `buffer` to + /// append an element to. Call `flush` with an index and a buffer to send the data downstream. + fn push_partitioned(&mut self, buffers: &mut [Self], index: I, flush: F) + where + I: FnMut(&Self::Item) -> usize, + F: FnMut(usize, &mut Self); +} + +impl PushPartitioned for Vec { + fn push_partitioned(&mut self, buffers: &mut [Self], mut index: I, mut flush: F) + where + I: FnMut(&Self::Item) -> usize, + F: FnMut(usize, &mut Self), + { + fn ensure_capacity(this: &mut Vec) { + let capacity = this.capacity(); + let desired_capacity = buffer::default_capacity::(); + if capacity < desired_capacity { + this.reserve(desired_capacity - capacity); + } + } + + for datum in self.drain(..) { + let index = index(&datum); + ensure_capacity(&mut buffers[index]); + buffers[index].push(datum); + if buffers[index].len() == buffers[index].capacity() { + flush(index, &mut buffers[index]); + } + } + } +} + +pub mod buffer { + //! Functionality related to calculating default buffer sizes + + /// The upper limit for buffers to allocate, size in bytes. [default_capacity] converts + /// this to size in elements. + pub const BUFFER_SIZE_BYTES: usize = 1 << 13; + + /// The maximum buffer capacity in elements. Returns a number between [BUFFER_SIZE_BYTES] + /// and 1, inclusively. + pub const fn default_capacity() -> usize { + let size = ::std::mem::size_of::(); + if size == 0 { + BUFFER_SIZE_BYTES + } else if size <= BUFFER_SIZE_BYTES { + BUFFER_SIZE_BYTES / size + } else { + 1 + } + } +} diff --git a/experiments/Cargo.toml b/experiments/Cargo.toml deleted file mode 100644 index 6a0453f18..000000000 --- a/experiments/Cargo.toml +++ /dev/null @@ -1,12 +0,0 @@ -[package] -name = "experiments" -version = "0.1.0" -edition = "2018" - -[dependencies] -criterion = "0.3.5" -timely = { path = "../timely" } - -[[bench]] -name = "exchange_bench" -harness = false diff --git a/experiments/benches/exchange_bench.rs b/experiments/benches/exchange_bench.rs deleted file mode 100644 index cf9337ca4..000000000 --- a/experiments/benches/exchange_bench.rs +++ /dev/null @@ -1,104 +0,0 @@ -extern crate timely; - -use std::fmt::{Display, Formatter}; -use std::time::{Duration, Instant}; - -use criterion::black_box; -use criterion::*; - -use timely::dataflow::operators::{Exchange, Input, Probe}; -use timely::dataflow::InputHandle; -use timely::{CommunicationConfig, Config, WorkerConfig}; - -#[derive(Clone)] -struct ExperimentConfig { - threads: usize, - batch: u64, -} - -impl Display for ExperimentConfig { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "threads={:2},batch={:5}", self.threads, self.batch) - } -} - -fn bench(c: &mut Criterion) { - let mut group = c.benchmark_group("exchange"); - for threads in [1, 2, 4, 8, 16] { - for shift in [0, 4, 8, 14] { - let params = ExperimentConfig { - threads, - batch: 1u64 << shift, - }; - group.bench_with_input( - BenchmarkId::new("Default", params.clone()), - ¶ms, - move |b, params| { - b.iter_custom(|iters| { - let config = Config::process(params.threads); - black_box(experiment_exchange( - config, - params.batch, - iters, - )) - }) - }, - ); - group.bench_with_input( - BenchmarkId::new("DefaultZero", params.clone()), - ¶ms, - move |b, params| { - b.iter_custom(|iters| { - let config = Config { - communication: CommunicationConfig::ProcessBinary(params.threads), - worker: WorkerConfig::default(), - }; - black_box(experiment_exchange( - config, - params.batch, - iters, - )) - }) - }, - ); - } - } -} - -fn experiment_exchange( - config: Config, - batch: u64, - rounds: u64, -) -> Duration { - timely::execute(config, move |worker| { - let mut input = InputHandle::new(); - let probe = worker.dataflow(|scope| scope.input_from(&mut input).exchange(|x| *x).probe()); - - let mut time = 0; - let timer = Instant::now(); - - let buffer = (0..batch).collect(); - let mut copy = Vec::new(); - - for _round in 0..rounds { - copy.clone_from(&buffer); - input.send_batch(&mut copy); - copy.clear(); - time += 1; - input.advance_to(time); - while probe.less_than(input.time()) { - worker.step(); - } - } - timer.elapsed() - }) - .unwrap() - .join() - .into_iter() - .next() - .unwrap() - .unwrap() -} - -criterion_group!(benches, bench); -criterion_main!(benches); diff --git a/kafkaesque/.gitignore b/kafkaesque/.gitignore deleted file mode 100644 index ff0d8477b..000000000 --- a/kafkaesque/.gitignore +++ /dev/null @@ -1,3 +0,0 @@ -/target -/.vscode -Cargo.lock diff --git a/kafkaesque/Cargo.toml b/kafkaesque/Cargo.toml deleted file mode 100644 index 4ba07f664..000000000 --- a/kafkaesque/Cargo.toml +++ /dev/null @@ -1,13 +0,0 @@ -[package] -name = "kafkaesque" -version = "0.12.0" -authors = ["Frank McSherry "] -edition = "2018" - -[dependencies] -clap="*" -abomonation="0.7" -timely = { path = "../timely" } - -[dependencies.rdkafka] -version = "0.23.0" diff --git a/kafkaesque/src/bin/capture_recv.rs b/kafkaesque/src/bin/capture_recv.rs deleted file mode 100644 index 862d44cbf..000000000 --- a/kafkaesque/src/bin/capture_recv.rs +++ /dev/null @@ -1,46 +0,0 @@ -use timely::dataflow::operators::Inspect; -use timely::dataflow::operators::capture::Replay; -use timely::dataflow::operators::Accumulate; - -use rdkafka::config::ClientConfig; - -use kafkaesque::EventConsumer; - -fn main() { - timely::execute_from_args(std::env::args(), |worker| { - - let topic = std::env::args().nth(1).unwrap(); - let source_peers = std::env::args().nth(2).unwrap().parse::().unwrap(); - let brokers = "localhost:9092"; - - // Create Kafka stuff. - let mut consumer_config = ClientConfig::new(); - consumer_config - .set("produce.offset.report", "true") - .set("auto.offset.reset", "smallest") - .set("group.id", "example") - .set("enable.auto.commit", "false") - .set("enable.partition.eof", "false") - .set("auto.offset.reset", "earliest") - .set("session.timeout.ms", "6000") - .set("bootstrap.servers", &brokers); - - // create replayers from disjoint partition of source worker identifiers. - let replayers = - (0 .. source_peers) - .filter(|i| i % worker.peers() == worker.index()) - .map(|i| { - let topic = format!("{}-{:?}", topic, i); - EventConsumer::<_,u64>::new(consumer_config.clone(), topic) - }) - .collect::>(); - - worker.dataflow::(|scope| { - replayers - .replay_into(scope) - .count() - .inspect(|x| println!("replayed: {:?}", x)) - ; - }) - }).unwrap(); // asserts error-free execution -} diff --git a/kafkaesque/src/bin/capture_send.rs b/kafkaesque/src/bin/capture_send.rs deleted file mode 100644 index ea2502507..000000000 --- a/kafkaesque/src/bin/capture_send.rs +++ /dev/null @@ -1,31 +0,0 @@ -use timely::dataflow::operators::ToStream; -use timely::dataflow::operators::capture::Capture; - -use rdkafka::config::ClientConfig; - -use kafkaesque::EventProducer; - -fn main() { - timely::execute_from_args(std::env::args(), |worker| { - - // target topic name. - let topic = std::env::args().nth(1).unwrap(); - let count = std::env::args().nth(2).unwrap().parse::().unwrap(); - let brokers = "localhost:9092"; - - // Create Kafka stuff. - let mut producer_config = ClientConfig::new(); - producer_config - .set("produce.offset.report", "true") - .set("bootstrap.servers", brokers); - - let topic = format!("{}-{:?}", topic, worker.index()); - let producer = EventProducer::new(producer_config, topic); - - worker.dataflow::(|scope| - (0 .. count) - .to_stream(scope) - .capture_into(producer) - ); - }).unwrap(); -} diff --git a/kafkaesque/src/bin/kafka_source.rs b/kafkaesque/src/bin/kafka_source.rs deleted file mode 100644 index f71f8df7f..000000000 --- a/kafkaesque/src/bin/kafka_source.rs +++ /dev/null @@ -1,63 +0,0 @@ -use timely::dataflow::operators::Inspect; - -use rdkafka::config::ClientConfig; -use rdkafka::consumer::{Consumer, BaseConsumer, DefaultConsumerContext}; - -fn main() { - - let mut args = ::std::env::args(); - args.next(); - - // Extract Kafka topic. - let topic = args.next().expect("Must specify a Kafka topic"); - let brokers = "localhost:9092"; - - // Create Kafka consumer configuration. - // Feel free to change parameters here. - let mut consumer_config = ClientConfig::new(); - consumer_config - .set("produce.offset.report", "true") - .set("auto.offset.reset", "smallest") - .set("group.id", "example") - .set("enable.auto.commit", "false") - .set("enable.partition.eof", "false") - .set("auto.offset.reset", "earliest") - .set("session.timeout.ms", "6000") - .set("bootstrap.servers", &brokers); - - timely::execute_from_args(args, move |worker| { - - // A dataflow for producing spans. - worker.dataflow::(|scope| { - - // Create a Kafka consumer. - let consumer : BaseConsumer = consumer_config.create().expect("Couldn't create consumer"); - consumer.subscribe(&[&topic]).expect("Failed to subscribe to topic"); - - let strings = - kafkaesque::source(scope, "KafkaStringSource", consumer, |bytes, capability, output| { - - // If the bytes are utf8, convert to string and send. - if let Ok(text) = std::str::from_utf8(bytes) { - output - .session(capability) - .give(text.to_string()); - } - - // We need some rule to advance timestamps ... - let time = *capability.time(); - capability.downgrade(&(time + 1)); - - // Indicate that we are not yet done. - false - }); - - strings.inspect(|x| println!("Observed: {:?}", x)); - - }); - - }).expect("Timely computation failed somehow"); - - println!("Hello, world!"); -} - diff --git a/kafkaesque/src/kafka_source.rs b/kafkaesque/src/kafka_source.rs deleted file mode 100644 index 0167913dd..000000000 --- a/kafkaesque/src/kafka_source.rs +++ /dev/null @@ -1,138 +0,0 @@ -use timely::Data; -use timely::dataflow::{Scope, Stream}; -use timely::dataflow::operators::Capability; -use timely::dataflow::operators::generic::OutputHandle; -use timely::dataflow::channels::pushers::Tee; - -use rdkafka::Message; -use rdkafka::consumer::{ConsumerContext, BaseConsumer}; - -/// Constructs a stream of data from a Kafka consumer. -/// -/// This method assembles a stream of data from a Kafka consumer and supplied -/// user logic for determining how to interpret the binary data Kafka supplies. -/// -/// The user logic is provided binary data as `&[u8]`, and mutable references to -/// a capability and an output handle, which the logic should use to produce data -/// if it is so inclined. The logic must return a bool indicating whether the stream -/// is complete (true indicates that the operator should cease data production and -/// shut down). -/// -/// # Examples -/// ```rust,no_run -/// use timely::dataflow::operators::Inspect; -/// -/// use rdkafka::Message; -/// use rdkafka::config::ClientConfig; -/// use rdkafka::consumer::{Consumer, BaseConsumer, DefaultConsumerContext}; -/// -/// fn main() { -/// -/// let mut args = ::std::env::args(); -/// args.next(); -/// -/// // Extract Kafka topic. -/// let topic = args.next().expect("Must specify a Kafka topic"); -/// let brokers = "localhost:9092"; -/// -/// // Create Kafka consumer configuration. -/// // Feel free to change parameters here. -/// let mut consumer_config = ClientConfig::new(); -/// consumer_config -/// .set("produce.offset.report", "true") -/// .set("auto.offset.reset", "smallest") -/// .set("group.id", "example") -/// .set("enable.auto.commit", "false") -/// .set("enable.partition.eof", "false") -/// .set("auto.offset.reset", "earliest") -/// .set("session.timeout.ms", "6000") -/// .set("bootstrap.servers", &brokers); -/// -/// timely::execute_from_args(args, move |worker| { -/// -/// // A dataflow for producing spans. -/// worker.dataflow::(|scope| { -/// -/// // Create a Kafka consumer. -/// let consumer : BaseConsumer = consumer_config.create().expect("Couldn't create consumer"); -/// consumer.subscribe(&[&topic]).expect("Failed to subscribe to topic"); -/// -/// let strings = -/// kafkaesque::source(scope, "KafkaStringSource", consumer, |bytes, capability, output| { -/// -/// // If the bytes are utf8, convert to string and send. -/// if let Ok(text) = std::str::from_utf8(bytes) { -/// output -/// .session(capability) -/// .give(text.to_string()); -/// } -/// -/// // We need some rule to advance timestamps ... -/// let time = *capability.time(); -/// capability.downgrade(&(time + 1)); -/// -/// // Indicate that we are not yet done. -/// false -/// }); -/// -/// strings.inspect(|x| println!("Observed: {:?}", x)); -/// -/// }); -/// -/// }).expect("Timely computation failed somehow"); -/// -/// println!("Hello, world!"); -/// } -/// ``` -pub fn kafka_source( - scope: &G, - name: &str, - consumer: BaseConsumer, - logic: L -) -> Stream -where - C: ConsumerContext+'static, - G: Scope, - D: Data, - L: Fn(&[u8], - &mut Capability, - &mut OutputHandle>) -> bool+'static, -{ - use timely::dataflow::operators::generic::source; - source(scope, name, move |capability, info| { - - let activator = scope.activator_for(&info.address[..]); - let mut cap = Some(capability); - - // define a closure to call repeatedly. - move |output| { - - // Act only if we retain the capability to send data. - let mut complete = false; - if let Some(mut capability) = cap.as_mut() { - - // Indicate that we should run again. - activator.activate(); - - // Repeatedly interrogate Kafka for [u8] messages. - // Cease only when Kafka stops returning new data. - // Could cease earlier, if we had a better policy. - while let Some(result) = consumer.poll(std::time::Duration::from_millis(0)) { - // If valid data back from Kafka - if let Ok(message) = result { - // Attempt to interpret bytes as utf8 ... - if let Some(payload) = message.payload() { - complete = logic(payload, &mut capability, output) || complete; - } - } - else { - println!("Kafka error"); - } - } - } - - if complete { cap = None; } - } - - }) -} \ No newline at end of file diff --git a/kafkaesque/src/lib.rs b/kafkaesque/src/lib.rs deleted file mode 100644 index 39ec51bc1..000000000 --- a/kafkaesque/src/lib.rs +++ /dev/null @@ -1,122 +0,0 @@ -use std::sync::Arc; -use std::sync::atomic::{AtomicIsize, Ordering}; - -use abomonation::Abomonation; -use timely::dataflow::operators::capture::event::{Event, EventPusher, EventIterator}; - -use rdkafka::Message; -use rdkafka::client::ClientContext; -use rdkafka::config::ClientConfig; -use rdkafka::producer::{BaseProducer, BaseRecord, ProducerContext, DeliveryResult}; -use rdkafka::consumer::{Consumer, BaseConsumer, DefaultConsumerContext}; - -use rdkafka::config::FromClientConfigAndContext; - -pub mod kafka_source; -pub use kafka_source::kafka_source as source; - -struct OutstandingCounterContext { - outstanding: Arc, -} - -impl ClientContext for OutstandingCounterContext { } - -impl ProducerContext for OutstandingCounterContext { - type DeliveryOpaque = (); - fn delivery(&self, _report: &DeliveryResult, _: Self::DeliveryOpaque) { - self.outstanding.fetch_sub(1, Ordering::SeqCst); - } -} - -impl OutstandingCounterContext { - pub fn new(counter: &Arc) -> Self { - OutstandingCounterContext { - outstanding: counter.clone() - } - } -} - -/// A wrapper for `W: Write` implementing `EventPusher`. -pub struct EventProducer { - topic: String, - buffer: Vec, - producer: BaseProducer, - counter: Arc, - phant: ::std::marker::PhantomData<(T,D)>, -} - -impl EventProducer { - /// Allocates a new `EventWriter` wrapping a supplied writer. - pub fn new(config: ClientConfig, topic: String) -> Self { - let counter = Arc::new(AtomicIsize::new(0)); - let context = OutstandingCounterContext::new(&counter); - let producer = BaseProducer::::from_config_and_context(&config, context).expect("Couldn't create producer"); - println!("allocating producer for topic {:?}", topic); - EventProducer { - topic: topic, - buffer: vec![], - producer: producer, - counter: counter, - phant: ::std::marker::PhantomData, - } - } -} - -impl EventPusher for EventProducer { - fn push(&mut self, event: Event) { - unsafe { ::abomonation::encode(&event, &mut self.buffer).expect("Encode failure"); } - // println!("sending {:?} bytes", self.buffer.len()); - self.producer.send::<(),[u8]>(BaseRecord::to(self.topic.as_str()).payload(&self.buffer[..])).unwrap(); - self.counter.fetch_add(1, Ordering::SeqCst); - self.producer.poll(std::time::Duration::from_millis(0)); - self.buffer.clear(); - } -} - -impl Drop for EventProducer { - fn drop(&mut self) { - while self.counter.load(Ordering::SeqCst) > 0 { - self.producer.poll(std::time::Duration::from_millis(10)); - } - } -} - -/// A Wrapper for `R: Read` implementing `EventIterator`. -pub struct EventConsumer { - consumer: BaseConsumer, - buffer: Vec, - phant: ::std::marker::PhantomData<(T,D)>, -} - -impl EventConsumer { - /// Allocates a new `EventReader` wrapping a supplied reader. - pub fn new(config: ClientConfig, topic: String) -> Self { - println!("allocating consumer for topic {:?}", topic); - let consumer : BaseConsumer = config.create().expect("Couldn't create consumer"); - consumer.subscribe(&[&topic]).expect("Failed to subscribe to topic"); - EventConsumer { - consumer: consumer, - buffer: Vec::new(), - phant: ::std::marker::PhantomData, - } - } -} - -impl EventIterator for EventConsumer { - fn next(&mut self) -> Option<&Event> { - if let Some(result) = self.consumer.poll(std::time::Duration::from_millis(0)) { - match result { - Ok(message) => { - self.buffer.clear(); - self.buffer.extend_from_slice(message.payload().unwrap()); - Some(unsafe { ::abomonation::decode::>(&mut self.buffer[..]).unwrap().0 }) - }, - Err(err) => { - println!("KafkaConsumer error: {:?}", err); - None - }, - } - } - else { None } - } -} diff --git a/mdbook/src/chapter_0/chapter_0_1.md b/mdbook/src/chapter_0/chapter_0_1.md index bb610cdab..0dc5c234a 100644 --- a/mdbook/src/chapter_0/chapter_0_1.md +++ b/mdbook/src/chapter_0/chapter_0_1.md @@ -2,7 +2,7 @@ Timely dataflow means to capture a large number of idioms, so it is a bit tricky to wrap together one example that shows off all of its features, but let's look at something that shows off some core functionality to give a taste. -The following complete program initializes a timely dataflow computation, in which participants can supply a stream of numbers which are exchanged between the workers based on their value. Workers print to the screen when they see numbers. You can also find this as [`examples/hello.rs`](https://github.com/frankmcsherry/timely-dataflow/blob/master/examples/hello.rs) in the [timely dataflow repository](https://github.com/frankmcsherry/timely-dataflow/tree/master/examples). +The following complete program initializes a timely dataflow computation, in which participants can supply a stream of numbers which are exchanged between the workers based on their value. Workers print to the screen when they see numbers. You can also find this as [`examples/hello.rs`](https://github.com/TimelyDataflow/timely-dataflow/blob/master/examples/hello.rs) in the [timely dataflow repository](https://github.com/TimelyDataflow/timely-dataflow/tree/master/examples). ```rust extern crate timely; @@ -44,7 +44,7 @@ We can run this program in a variety of configurations: with just a single worke To try this out yourself, first clone the timely dataflow repository using `git` ```ignore - Echidnatron% git clone https://github.com/frankmcsherry/timely-dataflow + Echidnatron% git clone https://github.com/TimelyDataflow/timely-dataflow Cloning into 'timely-dataflow'... remote: Counting objects: 14254, done. remote: Compressing objects: 100% (2267/2267), done. diff --git a/mdbook/src/chapter_2/chapter_2_1.md b/mdbook/src/chapter_2/chapter_2_1.md index 1c401c467..8f8fc6b94 100644 --- a/mdbook/src/chapter_2/chapter_2_1.md +++ b/mdbook/src/chapter_2/chapter_2_1.md @@ -34,4 +34,4 @@ There will be more to do to get data into `input`, and we aren't going to worry ## Other sources -There are other sources of input that are a bit more advanced. Once we learn how to create custom operators, the `source` method will allow us to create a custom operator with zero input streams and one output stream, which looks like a source of data (hence the name). There are also the `Capture` and `Replay` traits that allow us to exfiltrate the contents of a stream from one dataflow (using `capture_into`) and re-load it in another dataflow (using `replay_from`). +There are other sources of input that are a bit more advanced. Once we learn how to create custom operators, the `source` method will allow us to create a custom operator with zero input streams and one output stream, which looks like a source of data (hence the name). There are also the `Capture` and `Replay` traits that allow us to exfiltrate the contents of a stream from one dataflow (using `capture_into`) and re-load it in another dataflow (using `replay_into`). diff --git a/mdbook/src/chapter_2/chapter_2_4.md b/mdbook/src/chapter_2/chapter_2_4.md index ecbfe09cf..00ab0042d 100644 --- a/mdbook/src/chapter_2/chapter_2_4.md +++ b/mdbook/src/chapter_2/chapter_2_4.md @@ -118,7 +118,7 @@ The system is smart enough to notice when you downgrade and discard capabilities It may seem that we have only considered stateless operators, those that are only able to read from their inputs and immediately write to their outputs. But, you can have whatever state that you like, using the magic of Rust's closures. When we write a closure, it can capture ("close over") any state that is currently in scope, taking ownership of it. This is actually what we did up above with the capability. If that sounds too abstract, let's look at an example. -Our `unary` example from way back just incremented the value and passed it along. What if we wanted to only pass values larger than any value we have seen so far? We just define a variable `max` which we check and update as we would normally. Importantly, we should define it *outside* the closure we return, so that it persists across calls, and we need to use the `move` keyword so that the closure knows it is supposed to take ownership of the variable. +Our `unary` example from way back just incremented the value and passed it along. What if we wanted to only pass values larger than any value we have seen so far? We just define a variable `maximum` which we check and update as we would normally. Importantly, we should define it *outside* the closure we return, so that it persists across calls, and we need to use the `move` keyword so that the closure knows it is supposed to take ownership of the variable. ```rust extern crate timely; @@ -155,7 +155,7 @@ fn main() { This example just captures an integer, but you could just as easily define and capture ownership of a `HashMap`, or whatever complicated state you would like repeated access to. -Bear in mind that this example is probably a bit wrong, in that we update `max` without paying any attention to the times of the data that come past, and so we may report a sequence of values that doesn't seem to correspond with the sequence when sorted by time. Writing sane operators in the presence of batches of data at shuffled times requires more thought. Specifically, for an operator to put its input back in order it needs to understand which times it might see in the future, which was the reason we were so careful about those capabilities and is the subject of the next subsection. +Bear in mind that this example is probably a bit wrong, in that we update `maximum` without paying any attention to the times of the data that come past, and so we may report a sequence of values that doesn't seem to correspond with the sequence when sorted by time. Writing sane operators in the presence of batches of data at shuffled times requires more thought. Specifically, for an operator to put its input back in order it needs to understand which times it might see in the future, which was the reason we were so careful about those capabilities and is the subject of the next subsection. ### Frontiered operators @@ -163,7 +163,7 @@ Timely dataflow is constantly tracking the capabilities of operators throughout Specifically, each input has a `frontier` method which returns a `&[Timestamp]`, indicating a list of times such that any future time must be greater or equal to some element of the list. Often this list will just have a single element, indicating the "current" time, but as we get to more complicated forms of time ("partially ordered" time, if that means anything to you yet) we may need to report multiple incomparable timestamps. -This frontier information is invaluable for operators that must be sure that their output is correct and final before they send it as output. For our `max` example, we will want to wait to apply the new maximum until we are sure that we will not see any more elements at earlier times. That isn't to say we can't do anything with data we receive "early"; in the case of the maximum, each batch at a given time can be reduced down to just its maximum value, as all received values would be applied simultaneously. +This frontier information is invaluable for operators that must be sure that their output is correct and final before they send it as output. For our `maximum` example, we will want to wait to apply the new maximum until we are sure that we will not see any more elements at earlier times. That isn't to say we can't do anything with data we receive "early"; in the case of the maximum, each batch at a given time can be reduced down to just its maximum value, as all received values would be applied simultaneously. To make life easier for you, we've written a helper type called `Notificator` whose job in life is to help you keep track of times that you would like to send outputs, and to tell you when (according to your input frontiers) it is now safe to send the data. In fact, notificators do more by holding on to the *capabilities* for you, so that you can be sure that, even if you *don't* receive any more messages but just an indication that there will be none, you will still retain the ability to send your messages. diff --git a/mdbook/src/chapter_4/chapter_4_4.md b/mdbook/src/chapter_4/chapter_4_4.md index 56e2a6100..90e27bc62 100644 --- a/mdbook/src/chapter_4/chapter_4_4.md +++ b/mdbook/src/chapter_4/chapter_4_4.md @@ -1,6 +1,6 @@ # Capture and Replay -Timely dataflow has two fairly handy operators, `capture_into` and `replay_from`, that are great for transporting a timely dataflow stream from its native representation into data, and then back again. They are also a fine way to think about interoperating with other systems for streaming data. +Timely dataflow has two fairly handy operators, `capture_into` and `replay_into`, that are great for transporting a timely dataflow stream from its native representation into data, and then back again. They are also a fine way to think about interoperating with other systems for streaming data. ## Capturing Streams @@ -208,4 +208,4 @@ which just goes on and on, but which should produce 50 lines of text, with five There are several sorts of things you could capture into and replay from. In the `capture::events` module you will find two examples, a linked list and a binary serializer / deserializer (wrapper around `Write` and `Read` traits). The binary serializer is fairly general; we used it up above to wrap TCP streams. You could also write to files, or write to shared memory. However, be mindful that the serialization format (abomonation) is essentially the in-memory representation, and Rust makes no guarantees about the stability of such a representation across builds. -There is also [an in-progress Kafka adapter](https://github.com/frankmcsherry/timely-dataflow/tree/master/kafkaesque) available in the repository, which uses Kafka topics to store the binary representation of captured streams, which can then be replayed by any timely computation that can read them. This may be a while before it is sorted out, because Kafka seems to have a few quirks, but if you would like to help get in touch. +There is also [an in-progress Kafka adapter](https://github.com/TimelyDataflow/timely-dataflow/tree/master/kafkaesque) available in the repository, which uses Kafka topics to store the binary representation of captured streams, which can then be replayed by any timely computation that can read them. This may be a while before it is sorted out, because Kafka seems to have a few quirks, but if you would like to help get in touch. diff --git a/mdbook/src/chapter_5/chapter_5_3.md b/mdbook/src/chapter_5/chapter_5_3.md new file mode 100644 index 000000000..5cd38763e --- /dev/null +++ b/mdbook/src/chapter_5/chapter_5_3.md @@ -0,0 +1,76 @@ +# Containers + +Timely's core isn't tied to a specific representation of data that flows along dataflow edges. +While some convenience operators make assumptions about the type of batches, the core infrastructure is generic in what containers are exchanged. +This section explains what containers are and what their contract with the Timely APIs is. + +Many parts of Timely assume that data is organized into `Vec`, i.e., batches store data as consecutive elements in memory. +This abstractions works well for many cases but precludes some advanced techniques, such as transferring translated or columnar data between operators. +With the container abstraction, Timely specifies a minimal interface it requires tracking progress and provide data to operators. + +## Core operators + +In Timely, we provide a set of `Core` operators that are generic on the container type they can handle. +In most cases, the `Core` operators are a immediate generalization of their non-core variant, providing the semantically equivalent functionality. + +## Limitations + +A challenge when genericizing Timely operators is that all interfaces need to be declared independent of a concrete type, for example as part of a trait. +For this reason, Timely doesn't currently support operators that require knowledge of the elements of a container or how to partition a container, with the only exception being the `Vec` type. + +## A custom container + +Let's walk through an example container that resembles a `Result` type, but moves the storage to within the result. + +```rust +extern crate timely_container; + +use timely_container::Container; + +#[derive(Clone)] +enum ResultContainer { + Ok(Vec), + Err(E), +} + +impl Default for ResultContainer { + fn default() -> Self { + Self::Ok(Default::default()) + } +} + +impl Container for ResultContainer { + type Item = Result; + + fn len(&self) -> usize { + match self { + ResultContainer::Ok(data) => data.len(), + ResultContainer::Err(_) => 1, + } + } + + fn is_empty(&self) -> bool { + match self { + ResultContainer::Ok(data) => data.is_empty(), + ResultContainer::Err(_) => false, + } + } + + fn capacity(&self) -> usize { + match self { + ResultContainer::Ok(data) => data.capacity(), + ResultContainer::Err(_) => 1, + } + } + + fn clear(&mut self) { + match self { + ResultContainer::Ok(data) => data.clear(), + ResultContainer::Err(_) => {}, + } + } +} +``` + +The type can either store a vector of data, or a single error. +Its length is the length of the vector, or 1 if it represents an error. diff --git a/release-plz.toml b/release-plz.toml new file mode 100644 index 000000000..cdc1088fc --- /dev/null +++ b/release-plz.toml @@ -0,0 +1,12 @@ +[workspace] +# disable the changelog for all packages +changelog_update = false + +publish = false + +[[package]] +name = "timely" +# enable the changelog for this package +changelog_update = true +# set the path of the changelog to the root of the repository +changelog_path = "./CHANGELOG.md" diff --git a/timely/Cargo.toml b/timely/Cargo.toml index 831239389..9d8b68aad 100644 --- a/timely/Cargo.toml +++ b/timely/Cargo.toml @@ -29,9 +29,9 @@ abomonation_derive = "0.5" timely_bytes = { path = "../bytes", version = "0.12" } timely_logging = { path = "../logging", version = "0.12" } timely_communication = { path = "../communication", version = "0.12", default-features = false } +timely_container = { path = "../container", version = "0.12" } crossbeam-channel = "0.5.0" -futures-util = "0.3" [dev-dependencies] # timely_sort="0.1.6" -rand="0.4" +rand = { version = "0.8", features = ["small_rng"] } diff --git a/timely/examples/bfs.rs b/timely/examples/bfs.rs index 35cb52f8b..0dc2d6987 100644 --- a/timely/examples/bfs.rs +++ b/timely/examples/bfs.rs @@ -3,7 +3,7 @@ extern crate timely; use std::collections::HashMap; -use rand::{Rng, SeedableRng, StdRng}; +use rand::{Rng, SeedableRng, rngs::SmallRng}; use timely::dataflow::operators::{ToStream, Concat, Feedback, ConnectLoop}; use timely::dataflow::operators::generic::operator::Operator; @@ -21,8 +21,7 @@ fn main() { let index = worker.index(); let peers = worker.peers(); - let seed: &[_] = &[1, 2, 3, index]; - let mut rng: StdRng = SeedableRng::from_seed(seed); + let mut rng: SmallRng = SeedableRng::seed_from_u64(index as u64); // pending edges and node updates. let mut edge_list = Vec::new(); @@ -41,7 +40,7 @@ fn main() { // generate part of a random graph. let graph = (0..edges / peers) - .map(move |_| (rng.gen_range(0u32, nodes as u32), rng.gen_range(0u32, nodes as u32))) + .map(move |_| (rng.gen_range(0..nodes as u32), rng.gen_range(0..nodes as u32))) .to_stream(scope); // define a loop variable, for the (node, worker) pairs. diff --git a/timely/examples/hashjoin.rs b/timely/examples/hashjoin.rs index 27be5eece..ad8ef8809 100644 --- a/timely/examples/hashjoin.rs +++ b/timely/examples/hashjoin.rs @@ -3,7 +3,7 @@ extern crate timely; use std::collections::HashMap; -use rand::{Rng, SeedableRng, StdRng}; +use rand::{Rng, SeedableRng, rngs::SmallRng}; use timely::dataflow::*; use timely::dataflow::operators::{Input, Probe}; @@ -79,8 +79,7 @@ fn main() { .probe_with(&mut probe); }); - let seed: &[_] = &[1, 2, 3, index]; - let mut rng: StdRng = SeedableRng::from_seed(seed); + let mut rng: SmallRng = SeedableRng::seed_from_u64(index as u64); let timer = std::time::Instant::now(); @@ -90,8 +89,8 @@ fn main() { // Send some amount of data, no more than `batch`. let to_send = std::cmp::min(batch, vals/peers - sent); for _ in 0 .. to_send { - input1.send((rng.gen_range(0, keys), rng.gen_range(0, keys))); - input2.send((rng.gen_range(0, keys), rng.gen_range(0, keys))); + input1.send((rng.gen_range(0..keys), rng.gen_range(0..keys))); + input2.send((rng.gen_range(0..keys), rng.gen_range(0..keys))); } sent += to_send; diff --git a/timely/examples/pagerank.rs b/timely/examples/pagerank.rs index 136fb5a10..0a2da2351 100644 --- a/timely/examples/pagerank.rs +++ b/timely/examples/pagerank.rs @@ -2,7 +2,7 @@ extern crate rand; extern crate timely; use std::collections::HashMap; -use rand::{Rng, SeedableRng, StdRng}; +use rand::{Rng, SeedableRng, rngs::SmallRng}; use timely::dataflow::{InputHandle, ProbeHandle}; use timely::dataflow::operators::{Feedback, ConnectLoop, Probe}; @@ -161,12 +161,11 @@ fn main() { let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap(); let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap(); - let seed: &[_] = &[1, 2, 3, worker.index()]; - let mut rng1: StdRng = SeedableRng::from_seed(seed); - let mut rng2: StdRng = SeedableRng::from_seed(seed); + let mut rng1: SmallRng = SeedableRng::seed_from_u64(worker.index() as u64); + let mut rng2: SmallRng = SeedableRng::seed_from_u64(worker.index() as u64); for _ in 0 .. edges / worker.peers() { - input.send(((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), 1)); + input.send(((rng1.gen_range(0..nodes), rng1.gen_range(0..nodes)), 1)); } input.advance_to(1); @@ -176,8 +175,8 @@ fn main() { } for i in 1 .. 1000 { - input.send(((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), 1)); - input.send(((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)), -1)); + input.send(((rng1.gen_range(0..nodes), rng1.gen_range(0..nodes)), 1)); + input.send(((rng2.gen_range(0..nodes), rng2.gen_range(0..nodes)), -1)); input.advance_to(i + 1); while probe.less_than(input.time()) { worker.step(); diff --git a/timely/examples/rc.rs b/timely/examples/rc.rs index 18f6fe972..f79ef7472 100644 --- a/timely/examples/rc.rs +++ b/timely/examples/rc.rs @@ -8,7 +8,7 @@ use abomonation::Abomonation; #[derive(Debug, Clone)] pub struct Test { - field: Rc, + _field: Rc, } impl Abomonation for Test { @@ -32,7 +32,7 @@ fn main() { // introduce data and watch! for round in 0..10 { - input.send(Test { field: Rc::new(round) } ); + input.send(Test { _field: Rc::new(round) } ); input.advance_to(round + 1); worker.step_while(|| probe.less_than(input.time())); } diff --git a/timely/examples/unionfind.rs b/timely/examples/unionfind.rs index e5af0f3fd..ab3cfa814 100644 --- a/timely/examples/unionfind.rs +++ b/timely/examples/unionfind.rs @@ -3,7 +3,7 @@ extern crate timely; use std::cmp::Ordering; -use rand::{Rng, SeedableRng, StdRng}; +use rand::{Rng, SeedableRng, rngs::SmallRng}; use timely::dataflow::*; use timely::dataflow::operators::{Input, Exchange, Probe}; @@ -34,11 +34,10 @@ fn main() { .probe_with(&mut probe); }); - let seed: &[_] = &[1, 2, 3, index]; - let mut rng: StdRng = SeedableRng::from_seed(seed); + let mut rng: SmallRng = SeedableRng::seed_from_u64(index as u64); for edge in 0..(edges / peers) { - input.send((rng.gen_range(0, nodes), rng.gen_range(0, nodes))); + input.send((rng.gen_range(0..nodes), rng.gen_range(0..nodes))); if edge % batch == (batch - 1) { let next = input.epoch() + 1; input.advance_to(next); diff --git a/timely/src/dataflow/channels/mod.rs b/timely/src/dataflow/channels/mod.rs index 7d174f083..84303a039 100644 --- a/timely/src/dataflow/channels/mod.rs +++ b/timely/src/dataflow/channels/mod.rs @@ -1,6 +1,7 @@ //! Structured communication between timely dataflow operators. use crate::communication::Push; +use crate::Container; /// A collection of types that may be pushed at. pub mod pushers; @@ -10,7 +11,10 @@ pub mod pullers; pub mod pact; /// The input to and output from timely dataflow communication channels. -pub type Bundle = crate::communication::Message>; +pub type BundleCore = crate::communication::Message>; + +/// The input to and output from timely dataflow communication channels specialized to vectors. +pub type Bundle = BundleCore>; /// A serializable representation of timestamped data. #[derive(Clone, Abomonation, Serialize, Deserialize)] @@ -18,7 +22,7 @@ pub struct Message { /// The timestamp associated with the message. pub time: T, /// The data in the message. - pub data: Vec, + pub data: D, /// The source worker. pub from: usize, /// A sequence number for this worker-to-worker stream. @@ -27,49 +31,26 @@ pub struct Message { impl Message { /// Default buffer size. + #[deprecated = "Use timely::buffer::default_capacity instead"] pub fn default_length() -> usize { - const MESSAGE_BUFFER_SIZE: usize = 1 << 13; - let size = std::mem::size_of::(); - if size == 0 { - // We could use usize::MAX here, but to avoid overflows we - // limit the default length for zero-byte types. - MESSAGE_BUFFER_SIZE - } else if size <= MESSAGE_BUFFER_SIZE { - MESSAGE_BUFFER_SIZE / size - } else { - 1 - } + crate::container::buffer::default_capacity::() } +} +impl Message { /// Creates a new message instance from arguments. - pub fn new(time: T, data: Vec, from: usize, seq: usize) -> Self { + pub fn new(time: T, data: D, from: usize, seq: usize) -> Self { Message { time, data, from, seq } } /// Forms a message, and pushes contents at `pusher`. Replaces `buffer` with what the pusher - /// leaves in place, or a new `Vec`. Note that the returned vector is always initialized with - /// a capacity of [Self::default_length] elements. - #[inline] - pub fn push_at>>(buffer: &mut Vec, time: T, pusher: &mut P) { - - Self::push_at_no_allocation(buffer, time, pusher); - - // Allocate a default buffer to avoid oddly sized or empty buffers - if buffer.capacity() != Self::default_length() { - *buffer = Vec::with_capacity(Self::default_length()); - } - } - - /// Forms a message, and pushes contents at `pusher`. Replaces `buffer` with what the pusher - /// leaves in place, or a new empty `Vec`. If the pusher leaves a vector with a capacity larger - /// than [Self::default_length], the vector is initialized with a new vector with - /// [Self::default_length] capacity. + /// leaves in place, or the container's default element. #[inline] - pub fn push_at_no_allocation>>(buffer: &mut Vec, time: T, pusher: &mut P) { + pub fn push_at>>(buffer: &mut D, time: T, pusher: &mut P) { let data = ::std::mem::take(buffer); let message = Message::new(time, data, 0, 0); - let mut bundle = Some(Bundle::from_typed(message)); + let mut bundle = Some(BundleCore::from_typed(message)); pusher.push(&mut bundle); @@ -79,10 +60,5 @@ impl Message { buffer.clear(); } } - - // Avoid memory leaks by buffers growing out of bounds - if buffer.capacity() > Self::default_length() { - *buffer = Vec::with_capacity(Self::default_length()); - } } } diff --git a/timely/src/dataflow/channels/pact.rs b/timely/src/dataflow/channels/pact.rs index 43b5e4279..976023a19 100644 --- a/timely/src/dataflow/channels/pact.rs +++ b/timely/src/dataflow/channels/pact.rs @@ -8,33 +8,41 @@ //! The progress tracking logic assumes that this number is independent of the pact used. use std::{fmt::{self, Debug}, marker::PhantomData}; +use timely_container::PushPartitioned; use crate::communication::{Push, Pull, Data}; use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller}; +use crate::Container; use crate::worker::AsWorker; use crate::dataflow::channels::pushers::Exchange as ExchangePusher; -use super::{Bundle, Message}; +use super::{BundleCore, Message}; use crate::logging::{TimelyLogger as Logger, MessagesEvent}; +use crate::progress::Timestamp; -/// A `ParallelizationContract` allocates paired `Push` and `Pull` implementors. -pub trait ParallelizationContract { +/// A `ParallelizationContractCore` allocates paired `Push` and `Pull` implementors. +pub trait ParallelizationContractCore { /// Type implementing `Push` produced by this pact. - type Pusher: Push>+'static; + type Pusher: Push>+'static; /// Type implementing `Pull` produced by this pact. - type Puller: Pull>+'static; + type Puller: Pull>+'static; /// Allocates a matched pair of push and pull endpoints implementing the pact. fn connect(self, allocator: &mut A, identifier: usize, address: &[usize], logging: Option) -> (Self::Pusher, Self::Puller); } +/// A `ParallelizationContractCore` specialized for `Vec` containers +/// TODO: Use trait aliases once stable. +pub trait ParallelizationContract: ParallelizationContractCore> { } +impl>> ParallelizationContract for P { } + /// A direct connection #[derive(Debug)] pub struct Pipeline; -impl ParallelizationContract for Pipeline { - type Pusher = LogPusher>>; - type Puller = LogPuller>>; +impl ParallelizationContractCore for Pipeline { + type Pusher = LogPusher>>; + type Puller = LogPuller>>; fn connect(self, allocator: &mut A, identifier: usize, address: &[usize], logging: Option) -> (Self::Pusher, Self::Puller) { let (pusher, puller) = allocator.pipeline::>(identifier, address); // // ignore `&mut A` and use thread allocator @@ -45,12 +53,15 @@ impl ParallelizationContract for Pipeline { } /// An exchange between multiple observers by data -pub struct Exchange { hash_func: F, phantom: PhantomData } +pub struct ExchangeCore { hash_func: F, phantom: PhantomData<(C, D)> } + +/// [ExchangeCore] specialized to vector-based containers. +pub type Exchange = ExchangeCore, D, F>; -implu64+'static> Exchange { +implu64+'static> ExchangeCore { /// Allocates a new `Exchange` pact from a distribution function. - pub fn new(func: F) -> Exchange { - Exchange { + pub fn new(func: F) -> ExchangeCore { + ExchangeCore { hash_func: func, phantom: PhantomData, } @@ -58,19 +69,21 @@ implu64+'static> Exchange { } // Exchange uses a `Box` because it cannot know what type of pushable will return from the allocator. -implu64+'static> ParallelizationContract for Exchange { - // TODO: The closure in the type prevents us from naming it. - // Could specialize `ExchangePusher` to a time-free version. - type Pusher = Box>>; - type Puller = Box>>; - fn connect(mut self, allocator: &mut A, identifier: usize, address: &[usize], logging: Option) -> (Self::Pusher, Self::Puller) { - let (senders, receiver) = allocator.allocate::>(identifier, address); +implu64+'static> ParallelizationContractCore for ExchangeCore +where + C: Data + Container + PushPartitioned, +{ + type Pusher = ExchangePusher>>>, F>; + type Puller = LogPuller>>>; + + fn connect(self, allocator: &mut A, identifier: usize, address: &[usize], logging: Option) -> (Self::Pusher, Self::Puller) { + let (senders, receiver) = allocator.allocate::>(identifier, address); let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, allocator.index(), i, identifier, logging.clone())).collect::>(); - (Box::new(ExchangePusher::new(senders, move |_, d| (self.hash_func)(d))), Box::new(LogPuller::new(receiver, allocator.index(), identifier, logging.clone()))) + (ExchangePusher::new(senders, self.hash_func), LogPuller::new(receiver, allocator.index(), identifier, logging.clone())) } } -impl Debug for Exchange { +impl Debug for ExchangeCore { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Exchange").finish() } @@ -78,7 +91,7 @@ impl Debug for Exchange { /// Wraps a `Message` pusher to provide a `Push<(T, Content)>`. #[derive(Debug)] -pub struct LogPusher>> { +pub struct LogPusher>> { pusher: P, channel: usize, counter: usize, @@ -88,7 +101,7 @@ pub struct LogPusher>> { logging: Option, } -impl>> LogPusher { +impl>> LogPusher { /// Allocates a new pusher. pub fn new(pusher: P, source: usize, target: usize, channel: usize, logging: Option) -> Self { LogPusher { @@ -103,9 +116,9 @@ impl>> LogPusher { } } -impl>> Push> for LogPusher { +impl>> Push> for LogPusher { #[inline] - fn push(&mut self, pair: &mut Option>) { + fn push(&mut self, pair: &mut Option>) { if let Some(bundle) = pair { self.counter += 1; @@ -134,7 +147,7 @@ impl>> Push> for LogPusher { /// Wraps a `Message` puller to provide a `Pull<(T, Content)>`. #[derive(Debug)] -pub struct LogPuller>> { +pub struct LogPuller>> { puller: P, channel: usize, index: usize, @@ -142,7 +155,7 @@ pub struct LogPuller>> { logging: Option, } -impl>> LogPuller { +impl>> LogPuller { /// Allocates a new `Puller`. pub fn new(puller: P, index: usize, channel: usize, logging: Option) -> Self { LogPuller { @@ -155,9 +168,9 @@ impl>> LogPuller { } } -impl>> Pull> for LogPuller { +impl>> Pull> for LogPuller { #[inline] - fn pull(&mut self) -> &mut Option> { + fn pull(&mut self) -> &mut Option> { let result = self.puller.pull(); if let Some(bundle) = result { let channel = self.channel; diff --git a/timely/src/dataflow/channels/pullers/counter.rs b/timely/src/dataflow/channels/pullers/counter.rs index 66a06c6ec..8f9bbbf08 100644 --- a/timely/src/dataflow/channels/pullers/counter.rs +++ b/timely/src/dataflow/channels/pullers/counter.rs @@ -3,33 +3,61 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::dataflow::channels::Bundle; +use crate::dataflow::channels::BundleCore; use crate::progress::ChangeBatch; use crate::communication::Pull; +use crate::Container; /// A wrapper which accounts records pulled past in a shared count map. -pub struct Counter>> { +pub struct Counter>> { pullable: P, consumed: Rc>>, phantom: ::std::marker::PhantomData, } -impl>> Counter { +/// A guard type that updates the change batch counts on drop +pub struct ConsumedGuard { + consumed: Rc>>, + time: Option, + len: usize, +} + +impl ConsumedGuard { + pub(crate) fn time(&self) -> &T { + &self.time.as_ref().unwrap() + } +} + +impl Drop for ConsumedGuard { + fn drop(&mut self) { + // SAFETY: we're in a Drop impl, so this runs at most once + let time = self.time.take().unwrap(); + self.consumed.borrow_mut().update(time, self.len as i64); + } +} + +impl>> Counter { /// Retrieves the next timestamp and batch of data. #[inline] - pub fn next(&mut self) -> Option<&mut Bundle> { + pub fn next(&mut self) -> Option<&mut BundleCore> { + self.next_guarded().map(|(_guard, bundle)| bundle) + } + + #[inline] + pub(crate) fn next_guarded(&mut self) -> Option<(ConsumedGuard, &mut BundleCore)> { if let Some(message) = self.pullable.pull() { - if message.data.len() > 0 { - self.consumed.borrow_mut().update(message.time.clone(), message.data.len() as i64); - Some(message) - } - else { None } + let guard = ConsumedGuard { + consumed: Rc::clone(&self.consumed), + time: Some(message.time.clone()), + len: message.data.len(), + }; + Some((guard, message)) } else { None } } } -impl>> Counter { +impl>> Counter { /// Allocates a new `Counter` from a boxed puller. pub fn new(pullable: P) -> Self { Counter { diff --git a/timely/src/dataflow/channels/pushers/buffer.rs b/timely/src/dataflow/channels/pushers/buffer.rs index 288d5ff88..d18a0f84d 100644 --- a/timely/src/dataflow/channels/pushers/buffer.rs +++ b/timely/src/dataflow/channels/pushers/buffer.rs @@ -1,44 +1,50 @@ //! Buffering and session mechanisms to provide the appearance of record-at-a-time sending, //! with the performance of batched sends. -use crate::dataflow::channels::{Bundle, Message}; +use crate::dataflow::channels::{Bundle, BundleCore, Message}; use crate::progress::Timestamp; use crate::dataflow::operators::Capability; use crate::communication::Push; +use crate::{Container, Data}; /// Buffers data sent at the same time, for efficient communication. /// /// The `Buffer` type should be used by calling `session` with a time, which checks whether /// data must be flushed and creates a `Session` object which allows sending at the given time. #[derive(Debug)] -pub struct Buffer>> { - time: Option, // the currently open time, if it is open - buffer: Vec, // a buffer for records, to send at self.time +pub struct BufferCore>> { + /// the currently open time, if it is open + time: Option, + /// a buffer for records, to send at self.time + buffer: D, pusher: P, } -impl>> Buffer where T: Eq+Clone { +/// A buffer specialized to vector-based containers. +pub type Buffer = BufferCore, P>; + +impl>> BufferCore where T: Eq+Clone { /// Creates a new `Buffer`. - pub fn new(pusher: P) -> Buffer { - Buffer { + pub fn new(pusher: P) -> Self { + Self { time: None, - buffer: Vec::with_capacity(Message::::default_length()), + buffer: Default::default(), pusher, } } /// Returns a `Session`, which accepts data to send at the associated time - pub fn session(&mut self, time: &T) -> Session { + pub fn session(&mut self, time: &T) -> Session { if let Some(true) = self.time.as_ref().map(|x| x != time) { self.flush(); } self.time = Some(time.clone()); Session { buffer: self } } /// Allocates a new `AutoflushSession` which flushes itself on drop. - pub fn autoflush_session(&mut self, cap: Capability) -> AutoflushSession where T: Timestamp { + pub fn autoflush_session(&mut self, cap: Capability) -> AutoflushSessionCore where T: Timestamp { if let Some(true) = self.time.as_ref().map(|x| x != cap.time()) { self.flush(); } self.time = Some(cap.time().clone()); - AutoflushSession { + AutoflushSessionCore { buffer: self, _capability: cap, } @@ -63,8 +69,26 @@ impl>> Buffer where T: Eq+Clone { } } + // Gives an entire container at a specific time. + fn give_container(&mut self, vector: &mut C) { + if !vector.is_empty() { + // flush to ensure fifo-ness + self.flush(); + + let time = self.time.as_ref().expect("Buffer::give_container(): time is None.").clone(); + Message::push_at(vector, time, &mut self.pusher); + } + } +} + +impl>> Buffer where T: Eq+Clone { // internal method for use by `Session`. + #[inline] fn give(&mut self, data: D) { + if self.buffer.capacity() < crate::container::buffer::default_capacity::() { + let to_reserve = crate::container::buffer::default_capacity::() - self.buffer.capacity(); + self.buffer.reserve(to_reserve); + } self.buffer.push(data); // assert!(self.buffer.capacity() == Message::::default_length()); if self.buffer.len() == self.buffer.capacity() { @@ -75,9 +99,7 @@ impl>> Buffer where T: Eq+Clone { // Gives an entire message at a specific time. fn give_vec(&mut self, vector: &mut Vec) { // flush to ensure fifo-ness - if !self.buffer.is_empty() { - self.flush(); - } + self.flush(); let time = self.time.as_ref().expect("Buffer::give_vec(): time is None.").clone(); Message::push_at(vector, time, &mut self.pusher); @@ -90,11 +112,18 @@ impl>> Buffer where T: Eq+Clone { /// The `Session` struct provides the user-facing interface to an operator output, namely /// the `Buffer` type. A `Session` wraps a session of output at a specified time, and /// avoids what would otherwise be a constant cost of checking timestamp equality. -pub struct Session<'a, T, D, P: Push>+'a> where T: Eq+Clone+'a, D: 'a { - buffer: &'a mut Buffer, +pub struct Session<'a, T, C: Container, P: Push>+'a> where T: Eq+Clone+'a, C: 'a { + buffer: &'a mut BufferCore, } -impl<'a, T, D, P: Push>+'a> Session<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +impl<'a, T, C: Container, P: Push>+'a> Session<'a, T, C, P> where T: Eq+Clone+'a, C: 'a { + /// Provide a container at the time specified by the [Session]. + pub fn give_container(&mut self, container: &mut C) { + self.buffer.give_container(container) + } +} + +impl<'a, T, D: Data, P: Push>>+'a> Session<'a, T, Vec, P> where T: Eq+Clone+'a, D: 'a { /// Provides one record at the time specified by the `Session`. #[inline] pub fn give(&mut self, data: D) { @@ -121,15 +150,18 @@ impl<'a, T, D, P: Push>+'a> Session<'a, T, D, P> where T: Eq+Clone } /// A session which will flush itself when dropped. -pub struct AutoflushSession<'a, T: Timestamp, D, P: Push>+'a> where - T: Eq+Clone+'a, D: 'a { +pub struct AutoflushSessionCore<'a, T: Timestamp, C: Container, P: Push>+'a> where + T: Eq+Clone+'a, C: 'a { /// A reference to the underlying buffer. - buffer: &'a mut Buffer, + buffer: &'a mut BufferCore, /// The capability being used to send the data. _capability: Capability, } -impl<'a, T: Timestamp, D, P: Push>+'a> AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +/// Auto-flush session specialized to vector-based containers. +pub type AutoflushSession<'a, T, D, P> = AutoflushSessionCore<'a, T, Vec, P>; + +impl<'a, T: Timestamp, D: Data, P: Push>>+'a> AutoflushSessionCore<'a, T, Vec, P> where T: Eq+Clone+'a, D: 'a { /// Transmits a single record. #[inline] pub fn give(&mut self, data: D) { @@ -151,7 +183,7 @@ impl<'a, T: Timestamp, D, P: Push>+'a> AutoflushSession<'a, T, D, P } } -impl<'a, T: Timestamp, D, P: Push>+'a> Drop for AutoflushSession<'a, T, D, P> where T: Eq+Clone+'a, D: 'a { +impl<'a, T: Timestamp, C: Container, P: Push>+'a> Drop for AutoflushSessionCore<'a, T, C, P> where T: Eq+Clone+'a, C: 'a { fn drop(&mut self) { self.buffer.cease(); } diff --git a/timely/src/dataflow/channels/pushers/counter.rs b/timely/src/dataflow/channels/pushers/counter.rs index c8d6dff72..59ccacf32 100644 --- a/timely/src/dataflow/channels/pushers/counter.rs +++ b/timely/src/dataflow/channels/pushers/counter.rs @@ -4,21 +4,25 @@ use std::marker::PhantomData; use std::rc::Rc; use std::cell::RefCell; -use crate::progress::ChangeBatch; -use crate::dataflow::channels::Bundle; +use crate::progress::{ChangeBatch, Timestamp}; +use crate::dataflow::channels::BundleCore; use crate::communication::Push; +use crate::Container; /// A wrapper which updates shared `produced` based on the number of records pushed. #[derive(Debug)] -pub struct Counter>> { +pub struct CounterCore>> { pushee: P, produced: Rc>>, phantom: PhantomData, } -impl Push> for Counter where T : Ord+Clone+'static, P: Push> { +/// A counter specialized to vector. +pub type Counter = CounterCore, P>; + +impl Push> for CounterCore where P: Push> { #[inline] - fn push(&mut self, message: &mut Option>) { + fn push(&mut self, message: &mut Option>) { if let Some(message) = message { self.produced.borrow_mut().update(message.time.clone(), message.data.len() as i64); } @@ -30,10 +34,10 @@ impl Push> for Counter where T : Ord+Clone+'stati } } -impl>> Counter where T : Ord+Clone+'static { +impl>> CounterCore where T : Ord+Clone+'static { /// Allocates a new `Counter` from a pushee and shared counts. - pub fn new(pushee: P) -> Counter { - Counter { + pub fn new(pushee: P) -> CounterCore { + CounterCore { pushee, produced: Rc::new(RefCell::new(ChangeBatch::new())), phantom: PhantomData, diff --git a/timely/src/dataflow/channels/pushers/exchange.rs b/timely/src/dataflow/channels/pushers/exchange.rs index aad867aa2..9ea271d31 100644 --- a/timely/src/dataflow/channels/pushers/exchange.rs +++ b/timely/src/dataflow/channels/pushers/exchange.rs @@ -1,64 +1,51 @@ //! The exchange pattern distributes pushed data between many target pushees. -use crate::Data; +use timely_container::PushPartitioned; +use crate::{Container, Data}; use crate::communication::Push; -use crate::dataflow::channels::{Bundle, Message}; +use crate::dataflow::channels::{BundleCore, Message}; // TODO : Software write combining /// Distributes records among target pushees according to a distribution function. -pub struct Exchange>, H: FnMut(&T, &D) -> u64> { +pub struct Exchange>, H: FnMut(&D) -> u64> { pushers: Vec

, - buffers: Vec>, + buffers: Vec, current: Option, hash_func: H, + phantom: std::marker::PhantomData, } -impl>, H: FnMut(&T, &D)->u64> Exchange { +impl>, H: FnMut(&D) -> u64> Exchange { /// Allocates a new `Exchange` from a supplied set of pushers and a distribution function. - pub fn new(pushers: Vec

, key: H) -> Exchange { + pub fn new(pushers: Vec

, key: H) -> Exchange { let mut buffers = vec![]; for _ in 0..pushers.len() { - buffers.push(Vec::new()); + buffers.push(Default::default()); } Exchange { pushers, hash_func: key, buffers, current: None, + phantom: std::marker::PhantomData, } } #[inline] fn flush(&mut self, index: usize) { if !self.buffers[index].is_empty() { if let Some(ref time) = self.current { - Message::push_at_no_allocation(&mut self.buffers[index], time.clone(), &mut self.pushers[index]); - } - } - } - - /// Push data partitioned according to an index function. - #[inline(always)] - fn push_partitioned usize>(&mut self, time: &T, data: &mut Vec, func: F) { - for datum in data.drain(..) { - let index = (func)((self.hash_func)(time, &datum)); - - // Ensure allocated buffers: If the buffer's capacity is less than its default - // capacity, increase the capacity such that it matches the default. - if self.buffers[index].capacity() < Message::::default_length() { - let to_reserve = Message::::default_length() - self.buffers[index].capacity(); - self.buffers[index].reserve(to_reserve); - } - self.buffers[index].push(datum); - if self.buffers[index].len() == self.buffers[index].capacity() { - self.flush(index); + Message::push_at(&mut self.buffers[index], time.clone(), &mut self.pushers[index]); } } } } -impl>, H: FnMut(&T, &D)->u64> Push> for Exchange { +impl>, H: FnMut(&D) -> u64> Push> for Exchange +where + C: PushPartitioned +{ #[inline(never)] - fn push(&mut self, message: &mut Option>) { + fn push(&mut self, message: &mut Option>) { // if only one pusher, no exchange if self.pushers.len() == 1 { self.pushers[0].push(message); @@ -77,15 +64,31 @@ impl>, H: FnMut(&T, &D)->u64> Push = Rc>>>>>; +type PushList = Rc>>>>>; /// Wraps a shared list of `Box` to forward pushes to. Owned by `Stream`. -pub struct Tee { - buffer: Vec, +pub struct TeeCore { + buffer: D, shared: PushList, } -impl Push> for Tee { +/// [TeeCore] specialized to `Vec`-based container. +pub type Tee = TeeCore>; + +impl Push> for TeeCore { #[inline] - fn push(&mut self, message: &mut Option>) { + fn push(&mut self, message: &mut Option>) { let mut pushers = self.shared.borrow_mut(); if let Some(message) = message { for index in 1..pushers.len() { - self.buffer.extend_from_slice(&message.data); + self.buffer.clone_from(&message.data); Message::push_at(&mut self.buffer, message.time.clone(), &mut pushers[index-1]); } } @@ -39,12 +42,12 @@ impl Push> for Tee { } } -impl Tee { +impl TeeCore { /// Allocates a new pair of `Tee` and `TeeHelper`. - pub fn new() -> (Tee, TeeHelper) { + pub fn new() -> (TeeCore, TeeHelper) { let shared = Rc::new(RefCell::new(Vec::new())); - let port = Tee { - buffer: Vec::with_capacity(Message::::default_length()), + let port = TeeCore { + buffer: Default::default(), shared: shared.clone(), }; @@ -52,16 +55,16 @@ impl Tee { } } -impl Clone for Tee { - fn clone(&self) -> Tee { - Tee { - buffer: Vec::with_capacity(self.buffer.capacity()), +impl Clone for TeeCore { + fn clone(&self) -> Self { + Self { + buffer: Default::default(), shared: self.shared.clone(), } } } -impl Debug for Tee +impl Debug for TeeCore where D: Debug, { @@ -86,7 +89,7 @@ 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) { + pub fn add_pusher>+'static>(&self, pusher: P) { self.shared.borrow_mut().push(Box::new(pusher)); } } diff --git a/timely/src/dataflow/mod.rs b/timely/src/dataflow/mod.rs index 54da3cb7e..a6a3c33a9 100644 --- a/timely/src/dataflow/mod.rs +++ b/timely/src/dataflow/mod.rs @@ -13,9 +13,10 @@ //! }); //! ``` -pub use self::stream::Stream; +pub use self::stream::{StreamCore, Stream}; pub use self::scopes::{Scope, ScopeParent}; +pub use self::operators::input::HandleCore as InputHandleCore; pub use self::operators::input::Handle as InputHandle; pub use self::operators::probe::Handle as ProbeHandle; diff --git a/timely/src/dataflow/operators/branch.rs b/timely/src/dataflow/operators/branch.rs index 5d2000a18..70e087abd 100644 --- a/timely/src/dataflow/operators/branch.rs +++ b/timely/src/dataflow/operators/branch.rs @@ -2,8 +2,8 @@ use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; -use crate::dataflow::{Scope, Stream}; -use crate::Data; +use crate::dataflow::{Scope, Stream, StreamCore}; +use crate::{Container, Data}; /// Extension trait for `Stream`. pub trait Branch { @@ -71,7 +71,7 @@ impl Branch for Stream { } /// Extension trait for `Stream`. -pub trait BranchWhen { +pub trait BranchWhen: Sized { /// Takes one input stream and splits it into two output streams. /// For each time, the supplied closure is called. If it returns true, /// the records for that will be sent to the second returned stream, otherwise @@ -91,17 +91,11 @@ pub trait BranchWhen { /// after_five.inspect(|x| println!("Times 5 and later: {:?}", x)); /// }); /// ``` - fn branch_when( - &self, - condition: impl Fn(&S::Timestamp) -> bool + 'static, - ) -> (Stream, Stream); + fn branch_when(&self, condition: impl Fn(&T) -> bool + 'static) -> (Self, Self); } -impl BranchWhen for Stream { - fn branch_when( - &self, - condition: impl Fn(&S::Timestamp) -> bool + 'static, - ) -> (Stream, Stream) { +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); @@ -110,19 +104,19 @@ impl BranchWhen for Stream { builder.build(move |_| { - let mut vector = Vec::new(); + let mut container = Default::default(); move |_frontiers| { let mut output1_handle = output1.activate(); let mut output2_handle = output2.activate(); input.for_each(|time, data| { - data.swap(&mut vector); + data.swap(&mut container); let mut out = if condition(&time.time()) { output2_handle.session(&time) } else { output1_handle.session(&time) }; - out.give_vec(&mut vector); + out.give_container(&mut container); }); } }); diff --git a/timely/src/dataflow/operators/capability.rs b/timely/src/dataflow/operators/capability.rs index 82009a95a..5caa9bce8 100644 --- a/timely/src/dataflow/operators/capability.rs +++ b/timely/src/dataflow/operators/capability.rs @@ -27,9 +27,11 @@ use std::cell::RefCell; use std::fmt::{self, Debug}; use crate::order::PartialOrder; +use crate::progress::Antichain; use crate::progress::Timestamp; use crate::progress::ChangeBatch; use crate::scheduling::Activations; +use crate::dataflow::channels::pullers::counter::ConsumedGuard; /// An internal trait expressing the capability to send messages with a given timestamp. pub trait CapabilityTrait { @@ -147,7 +149,7 @@ impl Capability { /// Attempts to downgrade the capability to one corresponding to `new_time`. /// - /// Returns a [`DowngradeError`] if `self.time` is not less or equal to `new_time`. + /// Returns a [DowngradeError] if `self.time` is not less or equal to `new_time`. pub fn try_downgrade(&mut self, new_time: &T) -> Result<(), DowngradeError> { if let Some(new_capability) = self.try_delayed(new_time) { *self = new_capability; @@ -222,38 +224,49 @@ impl Display for DowngradeError { impl Error for DowngradeError {} +/// A shared list of shared output capability buffers. type CapabilityUpdates = Rc>>>>>; -/// An unowned capability, which can be used but not retained. +/// An capability of an input port. Holding onto this capability will implicitly holds onto a +/// capability for all the outputs ports this input is connected to, after the connection summaries +/// have been applied. /// -/// The capability reference supplies a `retain(self)` method which consumes the reference -/// and turns it into an owned capability -pub struct CapabilityRef<'cap, T: Timestamp+'cap> { - time: &'cap T, +/// This input capability supplies a `retain_for_output(self)` method which consumes the input +/// capability and turns it into a [Capability] for a specific output port. +pub struct InputCapability { + /// Output capability buffers, for use in minting capabilities. internal: CapabilityUpdates, + /// Timestamp summaries for each output. + summaries: Rc>>>, + /// A drop guard that updates the consumed capability this InputCapability refers to on drop + consumed_guard: ConsumedGuard, } -impl<'cap, T: Timestamp+'cap> CapabilityTrait for CapabilityRef<'cap, T> { - fn time(&self) -> &T { self.time } +impl CapabilityTrait for InputCapability { + fn time(&self) -> &T { self.time() } fn valid_for_output(&self, query_buffer: &Rc>>) -> bool { - // let borrow = ; - self.internal.borrow().iter().any(|rc| Rc::ptr_eq(rc, query_buffer)) + let borrow = self.summaries.borrow(); + self.internal.borrow().iter().enumerate().any(|(index, rc)| { + // To be valid, the output buffer must match and the timestamp summary needs to be the default. + Rc::ptr_eq(rc, query_buffer) && borrow[index].len() == 1 && borrow[index][0] == Default::default() + }) } } -impl<'cap, T: Timestamp + 'cap> CapabilityRef<'cap, T> { +impl InputCapability { /// Creates a new capability reference at `time` while incrementing (and keeping a reference to) /// the provided [`ChangeBatch`]. - pub(crate) fn new(time: &'cap T, internal: CapabilityUpdates) -> Self { - CapabilityRef { - time, + pub(crate) fn new(internal: CapabilityUpdates, summaries: Rc>>>, guard: ConsumedGuard) -> Self { + InputCapability { internal, + summaries, + consumed_guard: guard, } } /// The timestamp associated with this capability. pub fn time(&self) -> &T { - self.time + self.consumed_guard.time() } /// Makes a new capability for a timestamp `new_time` greater or equal to the timestamp of @@ -266,15 +279,11 @@ impl<'cap, T: Timestamp + 'cap> CapabilityRef<'cap, T> { /// Delays capability for a specific output port. pub fn delayed_for_output(&self, new_time: &T, output_port: usize) -> Capability { - // TODO : Test operator summary? - if !self.time.less_equal(new_time) { - panic!("Attempted to delay {:?} to {:?}, which is not beyond the capability's time.", self, new_time); - } - if output_port < self.internal.borrow().len() { + use crate::progress::timestamp::PathSummary; + if self.summaries.borrow()[output_port].iter().flat_map(|summary| summary.results_in(self.time())).any(|time| time.less_equal(new_time)) { Capability::new(new_time.clone(), self.internal.borrow()[output_port].clone()) - } - else { - panic!("Attempted to acquire a capability for a non-existent output port."); + } else { + panic!("Attempted to delay to a time ({:?}) not greater or equal to the operators input-output summary ({:?}) applied to the capabilities time ({:?})", new_time, self.summaries.borrow()[output_port], self.time()); } } @@ -283,34 +292,39 @@ impl<'cap, T: Timestamp + 'cap> CapabilityRef<'cap, T> { /// This method produces an owned capability which must be dropped to release the /// capability. Users should take care that these capabilities are only stored for /// as long as they are required, as failing to drop them may result in livelock. + /// + /// This method panics if the timestamp summary to output zero strictly advances the time. pub fn retain(self) -> Capability { - // mint(self.time.clone(), self.internal) self.retain_for_output(0) } /// Transforms to an owned capability for a specific output port. + /// + /// This method panics if the timestamp summary to `output_port` strictly advances the time. pub fn retain_for_output(self, output_port: usize) -> Capability { - if output_port < self.internal.borrow().len() { - Capability::new(self.time.clone(), self.internal.borrow()[output_port].clone()) + use crate::progress::timestamp::PathSummary; + let self_time = self.time().clone(); + if self.summaries.borrow()[output_port].iter().flat_map(|summary| summary.results_in(&self_time)).any(|time| time.less_equal(&self_time)) { + Capability::new(self_time, self.internal.borrow()[output_port].clone()) } else { - panic!("Attempted to acquire a capability for a non-existent output port."); + panic!("Attempted to retain a time ({:?}) not greater or equal to the operators input-output summary ({:?}) applied to the capabilities time ({:?})", self_time, self.summaries.borrow()[output_port], self_time); } } } -impl<'cap, T: Timestamp> Deref for CapabilityRef<'cap, T> { +impl Deref for InputCapability { type Target = T; fn deref(&self) -> &T { - self.time + self.time() } } -impl<'cap, T: Timestamp> Debug for CapabilityRef<'cap, T> { +impl Debug for InputCapability { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("CapabilityRef") - .field("time", &self.time) + f.debug_struct("InputCapability") + .field("time", self.time()) .field("internal", &"...") .finish() } diff --git a/timely/src/dataflow/operators/capture/capture.rs b/timely/src/dataflow/operators/capture/capture.rs index f3f2181cb..c62b95417 100644 --- a/timely/src/dataflow/operators/capture/capture.rs +++ b/timely/src/dataflow/operators/capture/capture.rs @@ -5,19 +5,19 @@ //! and there are several default implementations, including a linked-list, Rust's MPSC //! queue, and a binary serializer wrapping any `W: Write`. -use crate::Data; -use crate::dataflow::{Scope, Stream}; +use crate::dataflow::{Scope, StreamCore}; 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; use crate::progress::ChangeBatch; use crate::progress::Timestamp; -use super::{Event, EventPusher}; +use super::{EventCore, EventPusherCore}; /// Capture a stream of timestamped data for later replay. -pub trait Capture { +pub trait Capture { /// Captures a stream of timestamped data for later replay. /// /// # Examples @@ -30,7 +30,7 @@ pub trait Capture { /// use std::sync::{Arc, Mutex}; /// use timely::dataflow::Scope; /// use timely::dataflow::operators::{Capture, ToStream, Inspect}; - /// use timely::dataflow::operators::capture::{EventLink, Replay, Extract}; + /// use timely::dataflow::operators::capture::{EventLinkCore, Replay, Extract}; /// /// // get send and recv endpoints, wrap send to share /// let (send, recv) = ::std::sync::mpsc::channel(); @@ -42,7 +42,7 @@ pub trait Capture { /// let send = send.lock().unwrap().clone(); /// /// // these are to capture/replay the stream. - /// let handle1 = Rc::new(EventLink::new()); + /// let handle1 = Rc::new(EventLinkCore::new()); /// let handle2 = Some(handle1.clone()); /// /// worker.dataflow::(|scope1| @@ -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 Stream { - 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)); @@ -131,7 +131,7 @@ impl Capture for Stream { if !progress.frontiers[0].is_empty() { // transmit any frontier progress. let to_send = ::std::mem::replace(&mut progress.frontiers[0], ChangeBatch::new()); - event_pusher.push(Event::Progress(to_send.into_inner())); + event_pusher.push(EventCore::Progress(to_send.into_inner())); } use crate::communication::message::RefOrMut; @@ -142,8 +142,8 @@ impl Capture for Stream { RefOrMut::Ref(reference) => (&reference.time, RefOrMut::Ref(&reference.data)), RefOrMut::Mut(reference) => (&reference.time, RefOrMut::Mut(&mut reference.data)), }; - let vector = data.replace(Vec::new()); - event_pusher.push(Event::Messages(time.clone(), vector)); + let vector = data.replace(Default::default()); + event_pusher.push(EventCore::Messages(time.clone(), vector)); } input.consumed().borrow_mut().drain_into(&mut progress.consumeds[0]); false diff --git a/timely/src/dataflow/operators/capture/event.rs b/timely/src/dataflow/operators/capture/event.rs index 31e3d6b0d..252b7361c 100644 --- a/timely/src/dataflow/operators/capture/event.rs +++ b/timely/src/dataflow/operators/capture/event.rs @@ -5,36 +5,56 @@ //! of timestamps. /// Data and progress events of the captured stream. -#[derive(Debug, Clone, Abomonation, Hash, Ord, PartialOrd, Eq, PartialEq)] -pub enum Event { +#[derive(Debug, Clone, Abomonation, Hash, Ord, PartialOrd, Eq, PartialEq, Deserialize, Serialize)] +pub enum EventCore { /// Progress received via `push_external_progress`. Progress(Vec<(T, i64)>), /// Messages received via the data stream. - Messages(T, Vec), + Messages(T, D), } -/// Iterates over contained `Event`. +/// Data and progress events of the captured stream, specialized to vector-based containers. +pub type Event = EventCore>; + +/// Iterates over contained `EventCore`. /// /// The `EventIterator` trait describes types that can iterate over references to events, /// and which can be used to replay a stream into a new timely dataflow computation. /// /// This method is not simply an iterator because of the lifetime in the result. -pub trait EventIterator { +pub trait EventIteratorCore { + /// Iterates over references to `EventCore` elements. + fn next(&mut self) -> Option<&EventCore>; +} + +/// A [EventIteratorCore] specialized to vector-based containers. +// TODO: use trait aliases once stable. +pub trait EventIterator: EventIteratorCore> { /// Iterates over references to `Event` elements. fn next(&mut self) -> Option<&Event>; } +impl>> EventIterator for E { + fn next(&mut self) -> Option<&Event> { + >::next(self) + } +} -/// Receives `Event` events. -pub trait EventPusher { +/// Receives `EventCore` events. +pub trait EventPusherCore { /// Provides a new `Event` to the pusher. - fn push(&mut self, event: Event); + fn push(&mut self, event: EventCore); } +/// A [EventPusherCore] specialized to vector-based containers. +// TODO: use trait aliases once stable. +pub trait EventPusher: EventPusherCore> {} +impl>> EventPusher for E {} + // implementation for the linked list behind a `Handle`. -impl EventPusher for ::std::sync::mpsc::Sender> { - fn push(&mut self, event: Event) { +impl EventPusherCore for ::std::sync::mpsc::Sender> { + fn push(&mut self, event: EventCore) { // NOTE: An Err(x) result just means "data not accepted" most likely // because the receiver is gone. No need to panic. let _ = self.send(event); @@ -47,37 +67,40 @@ pub mod link { use std::rc::Rc; use std::cell::RefCell; - use super::{Event, EventPusher, EventIterator}; + use super::{EventCore, EventPusherCore, EventIteratorCore}; - /// A linked list of Event. - pub struct EventLink { + /// A linked list of EventCore. + pub struct EventLinkCore { /// An event, if one exists. /// /// An event might not exist, if either we want to insert a `None` and have the output iterator pause, /// or in the case of the very first linked list element, which has no event when constructed. - pub event: Option>, + pub event: Option>, /// The next event, if it exists. - pub next: RefCell>>>, + pub next: RefCell>>>, } - impl EventLink { + /// A [EventLinkCore] specialized to vector-based containers. + pub type EventLink = EventLinkCore>; + + impl EventLinkCore { /// Allocates a new `EventLink`. - pub fn new() -> EventLink { - EventLink { event: None, next: RefCell::new(None) } + pub fn new() -> EventLinkCore { + EventLinkCore { event: None, next: RefCell::new(None) } } } // implementation for the linked list behind a `Handle`. - impl EventPusher for Rc> { - fn push(&mut self, event: Event) { - *self.next.borrow_mut() = Some(Rc::new(EventLink { event: Some(event), next: RefCell::new(None) })); + impl EventPusherCore for Rc> { + fn push(&mut self, event: EventCore) { + *self.next.borrow_mut() = Some(Rc::new(EventLinkCore { event: Some(event), next: RefCell::new(None) })); let next = self.next.borrow().as_ref().unwrap().clone(); *self = next; } } - impl EventIterator for Rc> { - fn next(&mut self) -> Option<&Event> { + impl EventIteratorCore for Rc> { + fn next(&mut self) -> Option<&EventCore> { let is_some = self.next.borrow().is_some(); if is_some { let next = self.next.borrow().as_ref().unwrap().clone(); @@ -91,7 +114,7 @@ pub mod link { } // Drop implementation to prevent stack overflow through naive drop impl. - impl Drop for EventLink { + impl Drop for EventLinkCore { fn drop(&mut self) { while let Some(link) = self.next.replace(None) { if let Ok(head) = Rc::try_unwrap(link) { @@ -101,7 +124,7 @@ pub mod link { } } - impl Default for EventLink { + impl Default for EventLinkCore { fn default() -> Self { Self::new() } @@ -109,10 +132,10 @@ pub mod link { #[test] fn avoid_stack_overflow_in_drop() { - let mut event1 = Rc::new(EventLink::<(),()>::new()); + let mut event1 = Rc::new(EventLinkCore::<(),()>::new()); let _event2 = event1.clone(); for _ in 0 .. 1_000_000 { - event1.push(Event::Progress(vec![])); + event1.push(EventCore::Progress(vec![])); } } } @@ -122,33 +145,36 @@ pub mod binary { use std::io::Write; use abomonation::Abomonation; - use super::{Event, EventPusher, EventIterator}; + use super::{EventCore, EventPusherCore, EventIteratorCore}; - /// A wrapper for `W: Write` implementing `EventPusher`. - pub struct EventWriter { + /// A wrapper for `W: Write` implementing `EventPusherCore`. + pub struct EventWriterCore { stream: W, phant: ::std::marker::PhantomData<(T,D)>, } - impl EventWriter { + /// [EventWriterCore] specialized to vector-based containers. + pub type EventWriter = EventWriterCore, W>; + + impl EventWriterCore { /// Allocates a new `EventWriter` wrapping a supplied writer. - pub fn new(w: W) -> EventWriter { - EventWriter { + pub fn new(w: W) -> Self { + Self { stream: w, phant: ::std::marker::PhantomData, } } } - impl EventPusher for EventWriter { - fn push(&mut self, event: Event) { + impl EventPusherCore for EventWriterCore { + fn push(&mut self, event: EventCore) { // TODO: `push` has no mechanism to report errors, so we `unwrap`. unsafe { ::abomonation::encode(&event, &mut self.stream).expect("Event abomonation/write failed"); } } } /// A Wrapper for `R: Read` implementing `EventIterator`. - pub struct EventReader { + pub struct EventReaderCore { reader: R, bytes: Vec, buff1: Vec, @@ -158,10 +184,13 @@ pub mod binary { phant: ::std::marker::PhantomData<(T,D)>, } - impl EventReader { + /// [EventReaderCore] specialized to vector-based containers. + pub type EventReader = EventReaderCore, R>; + + impl EventReaderCore { /// Allocates a new `EventReader` wrapping a supplied reader. - pub fn new(r: R) -> EventReader { - EventReader { + pub fn new(r: R) -> Self { + Self { reader: r, bytes: vec![0u8; 1 << 20], buff1: vec![], @@ -173,12 +202,12 @@ pub mod binary { } } - impl EventIterator for EventReader { - fn next(&mut self) -> Option<&Event> { + impl EventIteratorCore for EventReaderCore { + fn next(&mut self) -> Option<&EventCore> { // if we can decode something, we should just return it! :D - if unsafe { ::abomonation::decode::>(&mut self.buff1[self.consumed..]) }.is_some() { - let (item, rest) = unsafe { ::abomonation::decode::>(&mut self.buff1[self.consumed..]) }.unwrap(); + if unsafe { ::abomonation::decode::>(&mut self.buff1[self.consumed..]) }.is_some() { + let (item, rest) = unsafe { ::abomonation::decode::>(&mut self.buff1[self.consumed..]) }.unwrap(); self.consumed = self.valid - rest.len(); return Some(item); } diff --git a/timely/src/dataflow/operators/capture/extract.rs b/timely/src/dataflow/operators/capture/extract.rs index e17146394..dcf57ae3b 100644 --- a/timely/src/dataflow/operators/capture/extract.rs +++ b/timely/src/dataflow/operators/capture/extract.rs @@ -1,12 +1,14 @@ //! Traits and types for extracting captured timely dataflow streams. -use super::Event; +use super::EventCore; +use crate::Container; +use crate::Data; /// Supports extracting a sequence of timestamp and data. pub trait Extract { /// Converts `self` into a sequence of timestamped data. /// - /// Currently this is only implemented for `Receiver>`, and is used only + /// Currently this is only implemented for `Receiver>>`, and is used only /// to easily pull data out of a timely dataflow computation once it has completed. /// /// # Examples @@ -16,7 +18,7 @@ pub trait Extract { /// use std::sync::{Arc, Mutex}; /// use timely::dataflow::Scope; /// use timely::dataflow::operators::{Capture, ToStream, Inspect}; - /// use timely::dataflow::operators::capture::{EventLink, Replay, Extract}; + /// use timely::dataflow::operators::capture::{EventLinkCore, Replay, Extract}; /// /// // get send and recv endpoints, wrap send to share /// let (send, recv) = ::std::sync::mpsc::channel(); @@ -28,7 +30,7 @@ pub trait Extract { /// let send = send.lock().unwrap().clone(); /// /// // these are to capture/replay the stream. - /// let handle1 = Rc::new(EventLink::new()); + /// let handle1 = Rc::new(EventLinkCore::new()); /// let handle2 = Some(handle1.clone()); /// /// worker.dataflow::(|scope1| @@ -47,15 +49,9 @@ pub trait Extract { fn extract(self) -> Vec<(T, Vec)>; } -impl Extract for ::std::sync::mpsc::Receiver> { +impl Extract for ::std::sync::mpsc::Receiver>> { fn extract(self) -> Vec<(T, Vec)> { - let mut result = Vec::new(); - for event in self { - if let Event::Messages(time, data) = event { - result.push((time, data)); - } - } - result.sort_by(|x,y| x.0.cmp(&y.0)); + let mut result = self.extract_core(); let mut current = 0; for i in 1 .. result.len() { @@ -75,3 +71,61 @@ impl Extract for ::std::sync::mpsc::Receiver> { result } } + +/// Supports extracting a sequence of timestamp and data. +pub trait ExtractCore { + /// Converts `self` into a sequence of timestamped data. + /// + /// Currently this is only implemented for `Receiver>`, and is used only + /// to easily pull data out of a timely dataflow computation once it has completed. + /// + /// # Examples + /// + /// ```rust + /// use std::rc::Rc; + /// use std::sync::{Arc, Mutex}; + /// use timely::dataflow::Scope; + /// use timely::dataflow::operators::{Capture, ToStream, Inspect}; + /// use timely::dataflow::operators::capture::{EventLinkCore, Replay, ExtractCore}; + /// + /// // get send and recv endpoints, wrap send to share + /// let (send, recv) = ::std::sync::mpsc::channel(); + /// let send = Arc::new(Mutex::new(send)); + /// + /// timely::execute(timely::Config::thread(), move |worker| { + /// + /// // this is only to validate the output. + /// let send = send.lock().unwrap().clone(); + /// + /// // these are to capture/replay the stream. + /// let handle1 = Rc::new(EventLinkCore::new()); + /// let handle2 = Some(handle1.clone()); + /// + /// worker.dataflow::(|scope1| + /// (0..10).to_stream(scope1) + /// .capture_into(handle1) + /// ); + /// + /// worker.dataflow(|scope2| { + /// handle2.replay_into(scope2) + /// .capture_into(send) + /// }); + /// }).unwrap(); + /// + /// assert_eq!(recv.extract_core().into_iter().flat_map(|x| x.1).collect::>(), (0..10).collect::>()); + /// ``` + fn extract_core(self) -> Vec<(T, C)>; +} + +impl ExtractCore for ::std::sync::mpsc::Receiver> { + fn extract_core(self) -> Vec<(T, C)> { + let mut result = Vec::new(); + for event in self { + if let EventCore::Messages(time, data) = event { + result.push((time, data)); + } + } + result.retain(|x| !x.1.is_empty()); + result + } +} diff --git a/timely/src/dataflow/operators/capture/mod.rs b/timely/src/dataflow/operators/capture/mod.rs index 713b84c96..22d332ea0 100644 --- a/timely/src/dataflow/operators/capture/mod.rs +++ b/timely/src/dataflow/operators/capture/mod.rs @@ -22,10 +22,10 @@ //! use std::rc::Rc; //! use timely::dataflow::Scope; //! use timely::dataflow::operators::{Capture, ToStream, Inspect}; -//! use timely::dataflow::operators::capture::{EventLink, Replay}; +//! use timely::dataflow::operators::capture::{EventLinkCore, Replay}; //! //! timely::execute(timely::Config::thread(), |worker| { -//! let handle1 = Rc::new(EventLink::new()); +//! let handle1 = Rc::new(EventLinkCore::new()); //! let handle2 = Some(handle1.clone()); //! //! worker.dataflow::(|scope1| @@ -75,11 +75,11 @@ pub use self::capture::Capture; pub use self::replay::Replay; -pub use self::extract::Extract; -pub use self::event::{Event, EventPusher}; -pub use self::event::link::EventLink; -pub use self::event::binary::EventReader; -pub use self::event::binary::EventWriter; +pub use self::extract::{Extract, ExtractCore}; +pub use self::event::{Event, EventCore, EventPusher, EventPusherCore}; +pub use self::event::link::{EventLink, EventLinkCore}; +pub use self::event::binary::{EventReader, EventReaderCore}; +pub use self::event::binary::{EventWriter, EventWriterCore}; pub mod capture; pub mod replay; diff --git a/timely/src/dataflow/operators/capture/replay.rs b/timely/src/dataflow/operators/capture/replay.rs index 2492076af..2cc325419 100644 --- a/timely/src/dataflow/operators/capture/replay.rs +++ b/timely/src/dataflow/operators/capture/replay.rs @@ -38,20 +38,20 @@ //! allowing the replay to occur in a timely dataflow computation with more or fewer workers //! than that in which the stream was captured. -use crate::Data; -use crate::dataflow::{Scope, Stream}; -use crate::dataflow::channels::pushers::Counter as PushCounter; -use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; +use crate::dataflow::{Scope, StreamCore}; +use crate::dataflow::channels::pushers::CounterCore as PushCounter; +use crate::dataflow::channels::pushers::buffer::BufferCore as PushBuffer; use crate::dataflow::operators::generic::builder_raw::OperatorBuilder; use crate::progress::Timestamp; -use super::Event; -use super::event::EventIterator; +use super::EventCore; +use super::event::EventIteratorCore; +use crate::Container; /// Replay a capture stream into a scope with the same timestamp. -pub trait Replay : Sized { +pub trait Replay : Sized { /// Replays `self` into the provided scope, as a `Stream`. - fn replay_into>(self, scope: &mut S) -> Stream { + fn replay_into>(self, scope: &mut S) -> StreamCore { self.replay_core(scope, Some(std::time::Duration::new(0, 0))) } /// Replays `self` into the provided scope, as a `Stream'. @@ -59,13 +59,13 @@ pub trait Replay : Sized { /// The `period` argument allows the specification of a re-activation period, where the operator /// will re-activate itself every so often. The `None` argument instructs the operator not to /// re-activate itself.us - fn replay_core>(self, scope: &mut S, period: Option) -> Stream; + fn replay_core>(self, scope: &mut S, period: Option) -> StreamCore; } -impl Replay for I +impl Replay for I where I : IntoIterator, - ::Item: EventIterator+'static { - fn replay_core>(self, scope: &mut S, period: Option) -> Stream{ + ::Item: EventIteratorCore+'static { + fn replay_core>(self, scope: &mut S, period: Option) -> StreamCore{ let mut builder = OperatorBuilder::new("Replay".to_owned(), scope.clone()); @@ -77,6 +77,7 @@ where I : IntoIterator, let mut output = PushBuffer::new(PushCounter::new(targets)); let mut event_streams = self.into_iter().collect::>(); let mut started = false; + let mut allocation: C = Default::default(); builder.build( move |progress| { @@ -91,12 +92,13 @@ where I : IntoIterator, for event_stream in event_streams.iter_mut() { while let Some(event) = event_stream.next() { - match *event { - Event::Progress(ref vec) => { + match event { + EventCore::Progress(vec) => { progress.internals[0].extend(vec.iter().cloned()); }, - Event::Messages(ref time, ref data) => { - output.session(time).give_iterator(data.iter().cloned()); + EventCore::Messages(ref time, data) => { + allocation.clone_from(data); + output.session(time).give_container(&mut allocation); } } } diff --git a/timely/src/dataflow/operators/concat.rs b/timely/src/dataflow/operators/concat.rs index db5e0182a..449609f5b 100644 --- a/timely/src/dataflow/operators/concat.rs +++ b/timely/src/dataflow/operators/concat.rs @@ -1,12 +1,12 @@ //! Merges the contents of multiple streams. -use crate::Data; +use crate::Container; use crate::dataflow::channels::pact::Pipeline; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{StreamCore, Scope}; /// Merge the contents of two streams. -pub trait Concat { +pub trait Concat { /// Merge the contents of two streams. /// /// # Examples @@ -20,17 +20,17 @@ pub trait Concat { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn concat(&self, _: &Stream) -> Stream; + fn concat(&self, _: &StreamCore) -> StreamCore; } -impl Concat for Stream { - fn concat(&self, other: &Stream) -> Stream { - self.scope().concatenate(vec![self.clone(), other.clone()]) +impl Concat for StreamCore { + fn concat(&self, other: &StreamCore) -> StreamCore { + self.scope().concatenate([self.clone(), other.clone()]) } } /// Merge the contents of multiple streams. -pub trait Concatenate { +pub trait Concatenate { /// Merge the contents of multiple streams. /// /// # Examples @@ -47,25 +47,25 @@ pub trait Concatenate { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn concatenate(&self, sources: I) -> Stream + fn concatenate(&self, sources: I) -> StreamCore where - I: IntoIterator>; + I: IntoIterator>; } -impl Concatenate for Stream { - fn concatenate(&self, sources: I) -> Stream +impl Concatenate for StreamCore { + fn concatenate(&self, sources: I) -> StreamCore where - I: IntoIterator> + I: IntoIterator> { let clone = self.clone(); self.scope().concatenate(Some(clone).into_iter().chain(sources)) } } -impl Concatenate for G { - fn concatenate(&self, sources: I) -> Stream +impl Concatenate for G { + fn concatenate(&self, sources: I) -> StreamCore where - I: IntoIterator> + I: IntoIterator> { // create an operator builder. @@ -81,13 +81,13 @@ impl Concatenate for G { // build an operator that plays out all input data. builder.build(move |_capability| { - let mut vector = Vec::new(); + let mut vector = Default::default(); move |_frontier| { let mut output = output.activate(); for handle in handles.iter_mut() { handle.for_each(|time, data| { data.swap(&mut vector); - output.session(&time).give_vec(&mut vector); + output.session(&time).give_container(&mut vector); }) } } diff --git a/timely/src/dataflow/operators/enterleave.rs b/timely/src/dataflow/operators/enterleave.rs index 6153d7f50..24c8fa12e 100644 --- a/timely/src/dataflow/operators/enterleave.rs +++ b/timely/src/dataflow/operators/enterleave.rs @@ -21,22 +21,23 @@ use std::marker::PhantomData; +use crate::logging::{TimelyLogger, MessagesEvent}; use crate::progress::Timestamp; use crate::progress::timestamp::Refines; use crate::progress::{Source, Target}; use crate::order::Product; -use crate::Data; +use crate::{Container, Data}; use crate::communication::Push; -use crate::dataflow::channels::pushers::{Counter, Tee}; -use crate::dataflow::channels::{Bundle, Message}; +use crate::dataflow::channels::pushers::{CounterCore, TeeCore}; +use crate::dataflow::channels::{BundleCore, Message}; use crate::worker::AsWorker; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{StreamCore, Scope, Stream}; use crate::dataflow::scopes::{Child, ScopeParent}; use crate::dataflow::operators::delay::Delay; /// Extension trait to move a `Stream` into a child of its current `Scope`. -pub trait Enter, D: Data> { +pub trait Enter, C: Container> { /// Moves the `Stream` argument into a child of its current `Scope`. /// /// # Examples @@ -51,7 +52,7 @@ pub trait Enter, D: Data> { /// }); /// }); /// ``` - fn enter<'a>(&self, _: &Child<'a, G, T>) -> Stream, D>; + fn enter<'a>(&self, _: &Child<'a, G, T>) -> StreamCore, C>; } use crate::dataflow::scopes::child::Iterative; @@ -75,37 +76,46 @@ pub trait EnterAt { fn enter_at<'a, F:FnMut(&D)->T+'static>(&self, scope: &Iterative<'a, G, T>, initial: F) -> Stream, D> ; } -impl::Timestamp, T>, D>> EnterAt for E { +impl::Timestamp, T>, Vec>> EnterAt for E { fn enter_at<'a, F:FnMut(&D)->T+'static>(&self, scope: &Iterative<'a, G, T>, mut initial: F) -> Stream, D> { self.enter(scope).delay(move |datum, time| Product::new(time.clone().to_outer(), initial(datum))) } } -impl, D: Data> Enter for Stream { - fn enter<'a>(&self, scope: &Child<'a, G, T>) -> Stream, D> { +impl, C: Data+Container> 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) = TeeCore::::new(); let ingress = IngressNub { - targets: Counter::new(targets), - phantom: ::std::marker::PhantomData, + targets: CounterCore::new(targets), + phantom: PhantomData, activator: scope.activator_for(&scope.addr()), active: false, }; let produced = ingress.targets.produced().clone(); - let input = scope.subgraph.borrow_mut().new_input(produced); - let channel_id = scope.clone().new_identifier(); - self.connect_to(input, ingress, channel_id); - Stream::new(Source::new(0, input.port), registrar, scope.clone()) + + if let Some(logger) = scope.logging() { + let pusher = LogPusher::new(ingress, channel_id, scope.index(), logger); + self.connect_to(input, pusher, channel_id); + } else { + self.connect_to(input, ingress, channel_id); + } + + StreamCore::new( + Source::new(0, input.port), + registrar, + scope.clone(), + ) } } /// Extension trait to move a `Stream` to the parent of its current `Scope`. -pub trait Leave { +pub trait Leave { /// Moves a `Stream` to the parent of its current `Scope`. /// /// # Examples @@ -120,20 +130,28 @@ pub trait Leave { /// }); /// }); /// ``` - fn leave(&self) -> Stream; + fn leave(&self) -> StreamCore; } -impl<'a, G: Scope, D: Data, T: Timestamp+Refines> Leave for Stream, D> { - fn leave(&self) -> Stream { +impl<'a, G: Scope, D: Clone+Container, T: Timestamp+Refines> Leave for StreamCore, D> { + fn leave(&self) -> StreamCore { let scope = self.scope(); let output = scope.subgraph.borrow_mut().new_output(); - let (targets, registrar) = Tee::::new(); + let target = Target::new(0, output.port); + let (targets, registrar) = TeeCore::::new(); + let egress = EgressNub { targets, phantom: PhantomData }; let channel_id = scope.clone().new_identifier(); - self.connect_to(Target::new(0, output.port), EgressNub { targets, phantom: PhantomData }, channel_id); - Stream::new( + if let Some(logger) = scope.logging() { + let pusher = LogPusher::new(egress, channel_id, scope.index(), logger); + self.connect_to(target, pusher, channel_id); + } else { + self.connect_to(target, egress, channel_id); + } + + StreamCore::new( output, registrar, scope.parent, @@ -142,19 +160,19 @@ impl<'a, G: Scope, D: Data, T: Timestamp+Refines> Leave for } -struct IngressNub, TData: Data> { - targets: Counter>, +struct IngressNub, TData: Container> { + targets: CounterCore>, phantom: ::std::marker::PhantomData, activator: crate::scheduling::Activator, active: bool, } -impl, TData: Data> Push> for IngressNub { - fn push(&mut self, message: &mut Option>) { - if let Some(message) = message { +impl, TData: Container> Push> for IngressNub { + fn push(&mut self, element: &mut Option>) { + if let Some(message) = element { let outer_message = message.as_mut(); - let data = ::std::mem::replace(&mut outer_message.data, Vec::new()); - let mut inner_message = Some(Bundle::from_typed(Message::new(TInner::to_inner(outer_message.time.clone()), data, 0, 0))); + let data = ::std::mem::take(&mut outer_message.data); + let mut inner_message = Some(BundleCore::from_typed(Message::new(TInner::to_inner(outer_message.time.clone()), data, 0, 0))); self.targets.push(&mut inner_message); if let Some(inner_message) = inner_message { if let Some(inner_message) = inner_message.if_typed() { @@ -175,17 +193,17 @@ impl, TData: Data> Push, TData: Data> { - targets: Tee, + targets: TeeCore, phantom: PhantomData, } -impl Push> for EgressNub +impl Push> for EgressNub where TOuter: Timestamp, TInner: Timestamp+Refines, TData: Data { - fn push(&mut self, message: &mut Option>) { + fn push(&mut self, message: &mut Option>) { if let Some(message) = message { let inner_message = message.as_mut(); - let data = ::std::mem::replace(&mut inner_message.data, Vec::new()); - let mut outer_message = Some(Bundle::from_typed(Message::new(inner_message.time.clone().to_outer(), data, 0, 0))); + let data = ::std::mem::take(&mut inner_message.data); + let mut outer_message = Some(BundleCore::from_typed(Message::new(inner_message.time.clone().to_outer(), data, 0, 0))); self.targets.push(&mut outer_message); if let Some(outer_message) = outer_message { if let Some(outer_message) = outer_message.if_typed() { @@ -197,6 +215,61 @@ where TOuter: Timestamp, TInner: Timestamp+Refines, TData: Data { } } +/// A pusher that logs messages passing through it. +/// +/// This type performs the same function as the `LogPusher` and `LogPuller` types in +/// `timely::dataflow::channels::pact`. We need a special implementation for `enter`/`leave` +/// channels because those don't have a puller connected. Thus, this pusher needs to log both the +/// send and the receive `MessageEvent`. +struct LogPusher

{ + pusher: P, + channel: usize, + counter: usize, + index: usize, + logger: TimelyLogger, +} + +impl

LogPusher

{ + fn new(pusher: P, channel: usize, index: usize, logger: TimelyLogger) -> Self { + Self { + pusher, + channel, + counter: 0, + index, + logger, + } + } +} + +impl Push> for LogPusher

+where + D: Container, + P: Push>, +{ + fn push(&mut self, element: &mut Option>) { + if let Some(bundle) = element { + let send_event = MessagesEvent { + is_send: true, + channel: self.channel, + source: self.index, + target: self.index, + seq_no: self.counter, + length: bundle.data.len(), + }; + let recv_event = MessagesEvent { + is_send: false, + ..send_event + }; + + self.logger.log(send_event); + self.logger.log(recv_event); + self.counter += 1; + } + + self.pusher.push(element); + } +} + #[cfg(test)] mod test { /// Test that nested scopes with pass-through edges (no operators) correctly communicate progress. @@ -212,7 +285,7 @@ mod test { use crate::dataflow::operators::{Enter, Leave}; // initializes and runs a timely dataflow. - crate::execute_from_args(std::env::args(), |worker| { + crate::execute(crate::Config::process(3), |worker| { let index = worker.index(); let mut input = InputHandle::new(); @@ -245,4 +318,4 @@ mod test { }).unwrap(); } -} \ No newline at end of file +} diff --git a/timely/src/dataflow/operators/exchange.rs b/timely/src/dataflow/operators/exchange.rs index c106ce37f..1f603df25 100644 --- a/timely/src/dataflow/operators/exchange.rs +++ b/timely/src/dataflow/operators/exchange.rs @@ -1,12 +1,13 @@ //! Exchange records between workers. use crate::ExchangeData; -use crate::dataflow::channels::pact::Exchange as ExchangePact; -use crate::dataflow::{Stream, Scope}; +use crate::container::PushPartitioned; +use crate::dataflow::channels::pact::ExchangeCore; use crate::dataflow::operators::generic::operator::Operator; +use crate::dataflow::{Scope, StreamCore}; /// Exchange records between workers. -pub trait Exchange { +pub trait Exchange { /// Exchange records between workers. /// /// The closure supplied should map a reference to a record to a `u64`, @@ -22,18 +23,23 @@ pub trait Exchange { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn exchange(&self, route: impl Fn(&D)->u64+'static) -> Self; + fn exchange(&self, route: impl FnMut(&D) -> u64 + 'static) -> Self; } -// impl, D: ExchangeData> Exchange for Stream { -impl Exchange for Stream { - fn exchange(&self, route: impl Fn(&D)->u64+'static) -> Stream { - let mut vector = Vec::new(); - self.unary(ExchangePact::new(route), "Exchange", move |_,_| move |input, output| { - input.for_each(|time, data| { - data.swap(&mut vector); - output.session(&time).give_vec(&mut vector); - }); +impl Exchange for StreamCore +where + C: PushPartitioned + ExchangeData, + C::Item: ExchangeData, +{ + fn exchange(&self, route: impl FnMut(&C::Item) -> u64 + 'static) -> StreamCore { + let mut container = Default::default(); + self.unary(ExchangeCore::new(route), "Exchange", |_, _| { + move |input, output| { + input.for_each(|time, data| { + data.swap(&mut container); + output.session(&time).give_container(&mut container); + }); + } }) } } diff --git a/timely/src/dataflow/operators/feedback.rs b/timely/src/dataflow/operators/feedback.rs index e4f855bc9..a7eb90c65 100644 --- a/timely/src/dataflow/operators/feedback.rs +++ b/timely/src/dataflow/operators/feedback.rs @@ -1,14 +1,14 @@ //! Create cycles in a timely dataflow graph. -use crate::Data; +use crate::{Container, Data}; use crate::progress::{Timestamp, PathSummary}; use crate::progress::frontier::Antichain; use crate::order::Product; -use crate::dataflow::channels::pushers::Tee; +use crate::dataflow::channels::pushers::TeeCore; use crate::dataflow::channels::pact::Pipeline; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{StreamCore, Scope, Stream}; use crate::dataflow::scopes::child::Iterative; use crate::dataflow::operators::generic::builder_rc::OperatorBuilder; use crate::dataflow::operators::generic::OutputWrapper; @@ -37,6 +37,29 @@ pub trait Feedback { /// }); /// ``` fn feedback(&mut self, summary: ::Summary) -> (Handle, Stream); + + /// Creates a [StreamCore] and a [HandleCore] to later bind the source of that `Stream`. + /// + /// The resulting `Stream` will have its data defined by a future call to `connect_loop` with + /// its `Handle` passed as an argument. Data passed through the stream will have their + /// timestamps advanced by `summary`, and will be dropped if the result exceeds `limit`. + /// + /// # Examples + /// ``` + /// use timely::dataflow::Scope; + /// use timely::dataflow::operators::{Feedback, ConnectLoop, ToStream, Concat, Inspect, BranchWhen}; + /// + /// timely::example(|scope| { + /// // circulate 0..10 for 100 iterations. + /// let (handle, cycle) = scope.feedback_core::>(1); + /// (0..10).to_stream(scope) + /// .concat(&cycle) + /// .inspect(|x| println!("seen: {:?}", x)) + /// .branch_when(|t| t < &100).1 + /// .connect_loop(handle); + /// }); + /// ``` + fn feedback_core(&mut self, summary: ::Summary) -> (HandleCore, StreamCore); } /// Creates a `Stream` and a `Handle` to later bind the source of that `Stream`. @@ -64,27 +87,31 @@ pub trait LoopVariable<'a, G: Scope, T: Timestamp> { /// }); /// }); /// ``` - fn loop_variable(&mut self, summary: T::Summary) -> (Handle, D>, Stream, D>); + fn loop_variable(&mut self, summary: T::Summary) -> (HandleCore, D>, StreamCore, D>); } impl Feedback for G { fn feedback(&mut self, summary: ::Summary) -> (Handle, Stream) { + self.feedback_core(summary) + } + + fn feedback_core(&mut self, summary: ::Summary) -> (HandleCore, StreamCore) { let mut builder = OperatorBuilder::new("Feedback".to_owned(), self.clone()); let (output, stream) = builder.new_output(); - (Handle { builder, summary, output }, stream) + (HandleCore { builder, summary, output }, stream) } } impl<'a, G: Scope, T: Timestamp> LoopVariable<'a, G, T> for Iterative<'a, G, T> { - fn loop_variable(&mut self, summary: T::Summary) -> (Handle, D>, Stream, D>) { - self.feedback(Product::new(Default::default(), summary)) + fn loop_variable(&mut self, summary: T::Summary) -> (HandleCore, D>, StreamCore, D>) { + self.feedback_core(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 @@ -102,11 +129,11 @@ pub trait ConnectLoop { /// .connect_loop(handle); /// }); /// ``` - fn connect_loop(&self, _: Handle); + fn connect_loop(&self, _: HandleCore); } -impl ConnectLoop for Stream { - fn connect_loop(&self, helper: Handle) { +impl ConnectLoop for StreamCore { + fn connect_loop(&self, helper: HandleCore) { let mut builder = helper.builder; let summary = helper.summary; @@ -114,7 +141,7 @@ impl ConnectLoop for Stream { let mut input = builder.new_input_connection(self, Pipeline, vec![Antichain::from_elem(summary.clone())]); - let mut vector = Vec::new(); + let mut vector = Default::default(); builder.build(move |_capability| move |_frontier| { let mut output = output.activate(); input.for_each(|cap, data| { @@ -123,7 +150,7 @@ impl ConnectLoop for Stream { let new_cap = cap.delayed(&new_time); output .session(&new_cap) - .give_vec(&mut vector); + .give_container(&mut vector); } }); }); @@ -132,8 +159,11 @@ impl ConnectLoop for Stream { /// A handle used to bind the source of a loop variable. #[derive(Debug)] -pub struct Handle { +pub struct HandleCore { builder: OperatorBuilder, summary: ::Summary, - output: OutputWrapper>, + output: OutputWrapper>, } + +/// A `HandleCore` specialized for using `Vec` as container +pub type Handle = HandleCore>; diff --git a/timely/src/dataflow/operators/generic/builder_raw.rs b/timely/src/dataflow/operators/generic/builder_raw.rs index a4c151ea8..8e97492af 100644 --- a/timely/src/dataflow/operators/generic/builder_raw.rs +++ b/timely/src/dataflow/operators/generic/builder_raw.rs @@ -8,16 +8,15 @@ use std::default::Default; use std::rc::Rc; use std::cell::RefCell; -use crate::Data; - use crate::scheduling::{Schedule, Activations}; use crate::progress::{Source, Target}; use crate::progress::{Timestamp, Operate, operate::SharedProgress, Antichain}; -use crate::dataflow::{Stream, Scope}; -use crate::dataflow::channels::pushers::Tee; -use crate::dataflow::channels::pact::ParallelizationContract; +use crate::Container; +use crate::dataflow::{StreamCore, Scope}; +use crate::dataflow::channels::pushers::TeeCore; +use crate::dataflow::channels::pact::ParallelizationContractCore; use crate::dataflow::operators::generic::operator_info::OperatorInfo; /// Contains type-free information about the operator properties. @@ -106,17 +105,17 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &Stream, pact: P) -> P::Puller + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> P::Puller where - P: ParallelizationContract { + P: ParallelizationContractCore { let connection = vec![Antichain::from_elem(Default::default()); self.shape.outputs]; self.new_input_connection(stream, pact, connection) } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input_connection(&mut self, stream: &Stream, 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 { + P: ParallelizationContractCore { let channel_id = self.scope.new_identifier(); let logging = self.scope.logging(); @@ -132,18 +131,18 @@ impl OperatorBuilder { } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (Tee, Stream) { + pub fn new_output(&mut self) -> (TeeCore, 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, Stream) { + pub fn new_output_connection(&mut self, connection: Vec::Summary>>) -> (TeeCore, StreamCore) { - let (targets, registrar) = Tee::::new(); + let (targets, registrar) = TeeCore::::new(); let source = Source::new(self.index, self.shape.outputs); - let stream = Stream::new(source, registrar, self.scope.clone()); + let stream = StreamCore::new(source, registrar, self.scope.clone()); self.shape.outputs += 1; assert_eq!(self.shape.inputs, connection.len()); diff --git a/timely/src/dataflow/operators/generic/builder_rc.rs b/timely/src/dataflow/operators/generic/builder_rc.rs index 21eee25dd..ca80e3182 100644 --- a/timely/src/dataflow/operators/generic/builder_rc.rs +++ b/timely/src/dataflow/operators/generic/builder_rc.rs @@ -4,20 +4,19 @@ use std::rc::Rc; use std::cell::RefCell; use std::default::Default; -use crate::Data; - use crate::progress::{ChangeBatch, Timestamp}; use crate::progress::operate::SharedProgress; use crate::progress::frontier::{Antichain, MutableAntichain}; -use crate::dataflow::{Stream, Scope}; -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::Container; +use crate::dataflow::{Scope, StreamCore}; +use crate::dataflow::channels::pushers::TeeCore; +use crate::dataflow::channels::pushers::CounterCore as PushCounter; +use crate::dataflow::channels::pushers::buffer::BufferCore as PushBuffer; +use crate::dataflow::channels::pact::ParallelizationContractCore; use crate::dataflow::channels::pullers::Counter as PullCounter; use crate::dataflow::operators::capability::Capability; -use crate::dataflow::operators::generic::handles::{InputHandle, new_input_handle, OutputWrapper}; +use crate::dataflow::operators::generic::handles::{InputHandleCore, new_input_handle, OutputWrapper}; use crate::dataflow::operators::generic::operator_info::OperatorInfo; use crate::dataflow::operators::generic::builder_raw::OperatorShape; @@ -32,6 +31,8 @@ pub struct OperatorBuilder { frontier: Vec>, consumed: Vec>>>, internal: Rc>>>>>, + /// For each input, a shared list of summaries to each output. + summaries: Vec::Summary>>>>>, produced: Vec>>>, logging: Option, } @@ -46,6 +47,7 @@ impl OperatorBuilder { frontier: Vec::new(), consumed: Vec::new(), internal: Rc::new(RefCell::new(Vec::new())), + summaries: Vec::new(), produced: Vec::new(), logging, } @@ -57,9 +59,9 @@ impl OperatorBuilder { } /// Adds a new input to a generic operator builder, returning the `Pull` implementor to use. - pub fn new_input(&mut self, stream: &Stream, pact: P) -> InputHandle + pub fn new_input(&mut self, stream: &StreamCore, pact: P) -> InputHandleCore where - P: ParallelizationContract { + P: ParallelizationContractCore { let connection = vec![Antichain::from_elem(Default::default()); self.builder.shape().outputs()]; self.new_input_connection(stream, pact, connection) @@ -73,21 +75,24 @@ 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: &Stream, pact: P, connection: Vec::Summary>>) -> InputHandle + pub fn new_input_connection(&mut self, stream: &StreamCore, pact: P, connection: Vec::Summary>>) -> InputHandleCore where - P: ParallelizationContract { + P: ParallelizationContractCore { - let puller = self.builder.new_input_connection(stream, pact, connection); + let puller = self.builder.new_input_connection(stream, pact, connection.clone()); let input = PullCounter::new(puller); self.frontier.push(MutableAntichain::new()); self.consumed.push(input.consumed().clone()); - new_input_handle(input, self.internal.clone(), self.logging.clone()) + let shared_summary = Rc::new(RefCell::new(connection)); + self.summaries.push(shared_summary.clone()); + + new_input_handle(input, self.internal.clone(), shared_summary, self.logging.clone()) } /// Adds a new output to a generic operator builder, returning the `Push` implementor to use. - pub fn new_output(&mut self) -> (OutputWrapper>, Stream) { + 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) } @@ -100,9 +105,9 @@ 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_output_connection(&mut self, connection: Vec::Summary>>) -> (OutputWrapper>, Stream) { + pub fn new_output_connection(&mut self, connection: Vec::Summary>>) -> (OutputWrapper>, StreamCore) { - let (tee, stream) = self.builder.new_output_connection(connection); + let (tee, stream) = self.builder.new_output_connection(connection.clone()); let internal = Rc::new(RefCell::new(ChangeBatch::new())); self.internal.borrow_mut().push(internal.clone()); @@ -110,6 +115,10 @@ impl OperatorBuilder { let mut buffer = PushBuffer::new(PushCounter::new(tee)); self.produced.push(buffer.inner().produced().clone()); + for (summary, connection) in self.summaries.iter().zip(connection.into_iter()) { + summary.borrow_mut().push(connection.clone()); + } + (OutputWrapper::new(buffer, internal), stream) } @@ -224,8 +233,8 @@ mod tests { let mut builder = OperatorBuilder::new("Failure".to_owned(), scope.clone()); // let mut input = builder.new_input(stream, Pipeline); - let (mut output1, _stream1) = builder.new_output::<()>(); - let (mut output2, _stream2) = builder.new_output::<()>(); + let (mut output1, _stream1) = builder.new_output::>(); + let (mut output2, _stream2) = builder.new_output::>(); builder.build(move |capabilities| { move |_frontiers| { @@ -254,8 +263,8 @@ mod tests { let mut builder = OperatorBuilder::new("Failure".to_owned(), scope.clone()); // let mut input = builder.new_input(stream, Pipeline); - let (mut output1, _stream1) = builder.new_output::<()>(); - let (mut output2, _stream2) = builder.new_output::<()>(); + let (mut output1, _stream1) = builder.new_output::>(); + let (mut output2, _stream2) = builder.new_output::>(); builder.build(move |mut capabilities| { move |_frontiers| { diff --git a/timely/src/dataflow/operators/generic/handles.rs b/timely/src/dataflow/operators/generic/handles.rs index b9d69f634..eeddf5295 100644 --- a/timely/src/dataflow/operators/generic/handles.rs +++ b/timely/src/dataflow/operators/generic/handles.rs @@ -6,50 +6,63 @@ use std::rc::Rc; use std::cell::RefCell; -use crate::Data; +use crate::progress::Antichain; use crate::progress::Timestamp; use crate::progress::ChangeBatch; use crate::progress::frontier::MutableAntichain; use crate::dataflow::channels::pullers::Counter as PullCounter; -use crate::dataflow::channels::pushers::Counter as PushCounter; -use crate::dataflow::channels::pushers::buffer::{Buffer, Session}; -use crate::dataflow::channels::Bundle; +use crate::dataflow::channels::pushers::CounterCore as PushCounter; +use crate::dataflow::channels::pushers::buffer::{BufferCore, Session}; +use crate::dataflow::channels::BundleCore; use crate::communication::{Push, Pull, message::RefOrMut}; +use crate::Container; use crate::logging::TimelyLogger as Logger; -use crate::dataflow::operators::CapabilityRef; +use crate::dataflow::operators::InputCapability; use crate::dataflow::operators::capability::CapabilityTrait; /// Handle to an operator's input stream. -pub struct InputHandle>> { +pub struct InputHandleCore>> { pull_counter: PullCounter, internal: Rc>>>>>, + /// Timestamp summaries from this input to each output. + /// + /// Each timestamp received through this input may only produce output timestamps + /// greater or equal to the input timestamp subjected to at least one of these summaries. + summaries: Rc>>>, logging: Option, } +/// Handle to an operator's input stream, specialized to vectors. +pub type InputHandle = InputHandleCore, P>; + /// Handle to an operator's input stream and frontier. -pub struct FrontieredInputHandle<'a, T: Timestamp, D: 'a, P: Pull>+'a> { +pub struct FrontieredInputHandleCore<'a, T: Timestamp, D: Container+'a, P: Pull>+'a> { /// The underlying input handle. - pub handle: &'a mut InputHandle, + pub handle: &'a mut InputHandleCore, /// The frontier as reported by timely progress tracking. pub frontier: &'a MutableAntichain, } -impl<'a, T: Timestamp, D: Data, P: Pull>> InputHandle { +/// Handle to an operator's input stream and frontier, specialized to vectors. +pub type FrontieredInputHandle<'a, T, D, P> = FrontieredInputHandleCore<'a, T, Vec, P>; + +impl<'a, T: Timestamp, D: Container, P: Pull>> InputHandleCore { /// Reads the next input buffer (at some timestamp `t`) and a corresponding capability for `t`. /// The timestamp `t` of the input buffer can be retrieved by invoking `.time()` on the capability. /// Returns `None` when there's no more data available. #[inline] - pub fn next(&mut self) -> Option<(CapabilityRef, RefOrMut>)> { + pub fn next(&mut self) -> Option<(InputCapability, RefOrMut)> { let internal = &self.internal; - self.pull_counter.next().map(|bundle| { + let summaries = &self.summaries; + self.pull_counter.next_guarded().map(|(guard, bundle)| { match bundle.as_ref_or_mut() { RefOrMut::Ref(bundle) => { - (CapabilityRef::new(&bundle.time, internal.clone()), RefOrMut::Ref(&bundle.data)) + (InputCapability::new(internal.clone(), summaries.clone(), guard), RefOrMut::Ref(&bundle.data)) }, RefOrMut::Mut(bundle) => { - (CapabilityRef::new(&bundle.time, internal.clone()), RefOrMut::Mut(&mut bundle.data)) + (InputCapability::new(internal.clone(), summaries.clone(), guard), RefOrMut::Mut(&mut bundle.data)) }, } }) @@ -74,31 +87,22 @@ impl<'a, T: Timestamp, D: Data, P: Pull>> InputHandle { /// }); /// ``` #[inline] - pub fn for_each, RefOrMut>)>(&mut self, mut logic: F) { - // We inline `next()` so that we can use `self.logging` without cloning (and dropping) the logger. - let internal = &self.internal; - while let Some((cap, data)) = self.pull_counter.next().map(|bundle| { - match bundle.as_ref_or_mut() { - RefOrMut::Ref(bundle) => { - (CapabilityRef::new(&bundle.time, internal.clone()), RefOrMut::Ref(&bundle.data)) - }, - RefOrMut::Mut(bundle) => { - (CapabilityRef::new(&bundle.time, internal.clone()), RefOrMut::Mut(&mut bundle.data)) - }, - } - }) { - self.logging.as_mut().map(|l| l.log(crate::logging::GuardedMessageEvent { is_start: true })); + pub fn for_each, RefOrMut)>(&mut self, mut logic: F) { + let mut logging = self.logging.take(); + while let Some((cap, data)) = self.next() { + logging.as_mut().map(|l| l.log(crate::logging::GuardedMessageEvent { is_start: true })); logic(cap, data); - self.logging.as_mut().map(|l| l.log(crate::logging::GuardedMessageEvent { is_start: false })); + logging.as_mut().map(|l| l.log(crate::logging::GuardedMessageEvent { is_start: false })); } + self.logging = logging; } } -impl<'a, T: Timestamp, D: Data, P: Pull>+'a> FrontieredInputHandle<'a, T, D, P> { +impl<'a, T: Timestamp, D: Container, P: Pull>+'a> FrontieredInputHandleCore<'a, T, D, P> { /// Allocate a new frontiered input handle. - pub fn new(handle: &'a mut InputHandle, frontier: &'a MutableAntichain) -> Self { - FrontieredInputHandle { + pub fn new(handle: &'a mut InputHandleCore, frontier: &'a MutableAntichain) -> Self { + FrontieredInputHandleCore { handle, frontier, } @@ -108,7 +112,7 @@ impl<'a, T: Timestamp, D: Data, P: Pull>+'a> FrontieredInputHandle< /// The timestamp `t` of the input buffer can be retrieved by invoking `.time()` on the capability. /// Returns `None` when there's no more data available. #[inline] - pub fn next(&mut self) -> Option<(CapabilityRef, RefOrMut>)> { + pub fn next(&mut self) -> Option<(InputCapability, RefOrMut)> { self.handle.next() } @@ -131,7 +135,7 @@ impl<'a, T: Timestamp, D: Data, P: Pull>+'a> FrontieredInputHandle< /// }); /// ``` #[inline] - pub fn for_each, RefOrMut>)>(&mut self, logic: F) { + pub fn for_each, RefOrMut)>(&mut self, logic: F) { self.handle.for_each(logic) } @@ -142,16 +146,22 @@ impl<'a, T: Timestamp, D: Data, P: Pull>+'a> FrontieredInputHandle< } } -pub fn _access_pull_counter>>(input: &mut InputHandle) -> &mut PullCounter { +pub fn _access_pull_counter>>(input: &mut InputHandleCore) -> &mut PullCounter { &mut input.pull_counter } /// Constructs an input handle. /// Declared separately so that it can be kept private when `InputHandle` is re-exported. -pub fn new_input_handle>>(pull_counter: PullCounter, internal: Rc>>>>>, logging: Option) -> InputHandle { - InputHandle { +pub fn new_input_handle>>( + pull_counter: PullCounter, + internal: Rc>>>>>, + summaries: Rc>>>, + logging: Option +) -> InputHandleCore { + InputHandleCore { pull_counter, internal, + summaries, logging, } } @@ -162,14 +172,14 @@ pub fn new_input_handle>>(pull_counter: Pu /// than with an `OutputHandle`, whose methods ensure that capabilities are used and that the /// pusher is flushed (via the `cease` method) once it is no longer used. #[derive(Debug)] -pub struct OutputWrapper>> { - push_buffer: Buffer>, +pub struct OutputWrapper>> { + push_buffer: BufferCore>, internal_buffer: Rc>>, } -impl>> OutputWrapper { +impl>> OutputWrapper { /// Creates a new output wrapper from a push buffer. - pub fn new(push_buffer: Buffer>, internal_buffer: Rc>>) -> Self { + pub fn new(push_buffer: BufferCore>, internal_buffer: Rc>>) -> Self { OutputWrapper { push_buffer, internal_buffer, @@ -179,8 +189,8 @@ impl>> OutputWrapper { /// /// This method ensures that the only access to the push buffer is through the `OutputHandle` /// type which ensures the use of capabilities, and which calls `cease` when it is dropped. - pub fn activate(&mut self) -> OutputHandle { - OutputHandle { + pub fn activate(&mut self) -> OutputHandleCore { + OutputHandleCore { push_buffer: &mut self.push_buffer, internal_buffer: &self.internal_buffer, } @@ -189,12 +199,15 @@ impl>> OutputWrapper { /// Handle to an operator's output stream. -pub struct OutputHandle<'a, T: Timestamp, D: 'a, P: Push>+'a> { - push_buffer: &'a mut Buffer>, +pub struct OutputHandleCore<'a, T: Timestamp, C: Container+'a, P: Push>+'a> { + push_buffer: &'a mut BufferCore>, internal_buffer: &'a Rc>>, } -impl<'a, T: Timestamp, D, P: Push>> OutputHandle<'a, T, D, P> { +/// Handle specialized to `Vec`-based container. +pub type OutputHandle<'a, T, D, P> = OutputHandleCore<'a, T, Vec, P>; + +impl<'a, T: Timestamp, C: Container, P: Push>> OutputHandleCore<'a, T, C, 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 @@ -217,13 +230,18 @@ impl<'a, T: Timestamp, D, P: Push>> OutputHandle<'a, T, D, P> { /// }); /// }); /// ``` - pub fn session<'b, C: CapabilityTrait>(&'b mut self, cap: &'b C) -> Session<'b, T, D, PushCounter> where 'a: 'b { + pub fn session<'b, CT: CapabilityTrait>(&'b mut self, cap: &'b CT) -> Session<'b, T, C, PushCounter> where 'a: 'b { assert!(cap.valid_for_output(&self.internal_buffer), "Attempted to open output session with invalid capability"); self.push_buffer.session(cap.time()) } + + /// Flushes all pending data and indicate that no more data immediately follows. + pub fn cease(&mut self) { + self.push_buffer.cease(); + } } -impl<'a, T: Timestamp, D, P: Push>> Drop for OutputHandle<'a, T, D, P> { +impl<'a, T: Timestamp, C: Container, P: Push>> Drop for OutputHandleCore<'a, T, C, P> { fn drop(&mut self) { self.push_buffer.cease(); } diff --git a/timely/src/dataflow/operators/generic/mod.rs b/timely/src/dataflow/operators/generic/mod.rs index e22f0b548..e54f5e423 100644 --- a/timely/src/dataflow/operators/generic/mod.rs +++ b/timely/src/dataflow/operators/generic/mod.rs @@ -8,7 +8,7 @@ mod handles; mod notificator; mod operator_info; -pub use self::handles::{InputHandle, FrontieredInputHandle, OutputHandle, OutputWrapper}; +pub use self::handles::{InputHandle, InputHandleCore, FrontieredInputHandle, FrontieredInputHandleCore, OutputHandle, OutputHandleCore, OutputWrapper}; pub use self::notificator::{Notificator, FrontierNotificator}; pub use self::operator::{Operator, source}; diff --git a/timely/src/dataflow/operators/generic/operator.rs b/timely/src/dataflow/operators/generic/operator.rs index 53a0d247a..afb7a25d1 100644 --- a/timely/src/dataflow/operators/generic/operator.rs +++ b/timely/src/dataflow/operators/generic/operator.rs @@ -1,22 +1,21 @@ //! Methods to construct generic streaming and blocking unary operators. -use crate::dataflow::channels::pushers::Tee; -use crate::dataflow::channels::pact::ParallelizationContract; +use crate::dataflow::channels::pushers::TeeCore; +use crate::dataflow::channels::pact::ParallelizationContractCore; -use crate::dataflow::operators::generic::handles::{InputHandle, FrontieredInputHandle, OutputHandle}; +use crate::dataflow::operators::generic::handles::{InputHandleCore, FrontieredInputHandleCore, OutputHandleCore}; use crate::dataflow::operators::capability::Capability; -use crate::Data; - -use crate::dataflow::{Stream, Scope}; +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; /// Methods to construct generic streaming and blocking operators. -pub trait Operator { +pub trait Operator { /// Creates a new dataflow operator that partitions its input stream by a parallelization /// strategy `pact`, and repeatedly invokes `logic`, the function returned by the function passed as `constructor`. /// `logic` can read from the input stream, write to the output stream, and inspect the frontier at the input. @@ -56,13 +55,13 @@ pub trait Operator { /// }); /// } /// ``` - fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> Stream + fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> StreamCore where - D2: Data, + D2: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut FrontieredInputHandle, - &mut OutputHandle>)+'static, - P: ParallelizationContract; + L: FnMut(&mut FrontieredInputHandleCore, + &mut OutputHandleCore>)+'static, + P: ParallelizationContractCore; /// 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`. @@ -93,12 +92,12 @@ pub trait Operator { /// }); /// } /// ``` - fn unary_notify, - &mut OutputHandle>, + fn unary_notify, + &mut OutputHandleCore>, &mut Notificator)+'static, - P: ParallelizationContract> - (&self, pact: P, name: &str, init: impl IntoIterator, logic: L) -> Stream; + P: ParallelizationContractCore> + (&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`. @@ -128,13 +127,13 @@ pub trait Operator { /// }); /// }); /// ``` - fn unary(&self, pact: P, name: &str, constructor: B) -> Stream + fn unary(&self, pact: P, name: &str, constructor: B) -> StreamCore where - D2: Data, + D2: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut InputHandle, - &mut OutputHandle>)+'static, - P: ParallelizationContract; + L: FnMut(&mut InputHandleCore, + &mut OutputHandleCore>)+'static, + P: ParallelizationContractCore; /// 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`. @@ -186,16 +185,16 @@ pub trait Operator { /// } /// }).unwrap(); /// ``` - fn binary_frontier(&self, other: &Stream, pact1: P1, pact2: P2, name: &str, constructor: B) -> Stream + fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - D2: Data, - D3: Data, + D2: Container, + D3: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut FrontieredInputHandle, - &mut FrontieredInputHandle, - &mut OutputHandle>)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract; + L: FnMut(&mut FrontieredInputHandleCore, + &mut FrontieredInputHandleCore, + &mut OutputHandleCore>)+'static, + P1: ParallelizationContractCore, + P2: ParallelizationContractCore; /// 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`. @@ -242,15 +241,15 @@ pub trait Operator { /// } /// }).unwrap(); /// ``` - fn binary_notify, - &mut InputHandle, - &mut OutputHandle>, + fn binary_notify, + &mut InputHandleCore, + &mut OutputHandleCore>, &mut Notificator)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract> - (&self, other: &Stream, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, logic: L) -> Stream; + P1: ParallelizationContractCore, + P2: ParallelizationContractCore> + (&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`. @@ -286,16 +285,16 @@ pub trait Operator { /// }).inspect(|x| println!("{:?}", x)); /// }); /// ``` - fn binary(&self, other: &Stream, pact1: P1, pact2: P2, name: &str, constructor: B) -> Stream + fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - D2: Data, - D3: Data, + D2: Container, + D3: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut InputHandle, - &mut InputHandle, - &mut OutputHandle>)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract; + L: FnMut(&mut InputHandleCore, + &mut InputHandleCore, + &mut OutputHandleCore>)+'static, + P1: ParallelizationContractCore, + P2: ParallelizationContractCore; /// Creates a new dataflow operator that partitions its input stream by a parallelization /// strategy `pact`, and repeatedly invokes the function `logic` which can read from the input stream @@ -322,19 +321,19 @@ pub trait Operator { /// ``` fn sink(&self, pact: P, name: &str, logic: L) where - L: FnMut(&mut FrontieredInputHandle)+'static, - P: ParallelizationContract; + L: FnMut(&mut FrontieredInputHandleCore)+'static, + P: ParallelizationContractCore; } -impl Operator for Stream { +impl Operator for StreamCore { - fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> Stream + fn unary_frontier(&self, pact: P, name: &str, constructor: B) -> StreamCore where - D2: Data, + D2: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut FrontieredInputHandle, - &mut OutputHandle>)+'static, - P: ParallelizationContract { + L: FnMut(&mut FrontieredInputHandleCore, + &mut OutputHandleCore>)+'static, + P: ParallelizationContractCore { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); let operator_info = builder.operator_info(); @@ -347,7 +346,7 @@ impl Operator for Stream { let capability = capabilities.pop().unwrap(); let mut logic = constructor(capability, operator_info); move |frontiers| { - let mut input_handle = FrontieredInputHandle::new(&mut input, &frontiers[0]); + let mut input_handle = FrontieredInputHandleCore::new(&mut input, &frontiers[0]); let mut output_handle = output.activate(); logic(&mut input_handle, &mut output_handle); } @@ -356,12 +355,12 @@ impl Operator for Stream { stream } - fn unary_notify, - &mut OutputHandle>, + fn unary_notify, + &mut OutputHandleCore>, &mut Notificator)+'static, - P: ParallelizationContract> - (&self, pact: P, name: &str, init: impl IntoIterator, mut logic: L) -> Stream { + P: ParallelizationContractCore> + (&self, pact: P, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { self.unary_frontier(pact, name, move |capability, _info| { let mut notificator = FrontierNotificator::new(); @@ -378,13 +377,13 @@ impl Operator for Stream { }) } - fn unary(&self, pact: P, name: &str, constructor: B) -> Stream + fn unary(&self, pact: P, name: &str, constructor: B) -> StreamCore where - D2: Data, + D2: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut InputHandle, - &mut OutputHandle>)+'static, - P: ParallelizationContract { + L: FnMut(&mut InputHandleCore, + &mut OutputHandleCore>)+'static, + P: ParallelizationContractCore { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); let operator_info = builder.operator_info(); @@ -406,16 +405,16 @@ impl Operator for Stream { stream } - fn binary_frontier(&self, other: &Stream, pact1: P1, pact2: P2, name: &str, constructor: B) -> Stream + fn binary_frontier(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - D2: Data, - D3: Data, + D2: Container, + D3: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut FrontieredInputHandle, - &mut FrontieredInputHandle, - &mut OutputHandle>)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract { + L: FnMut(&mut FrontieredInputHandleCore, + &mut FrontieredInputHandleCore, + &mut OutputHandleCore>)+'static, + P1: ParallelizationContractCore, + P2: ParallelizationContractCore { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); let operator_info = builder.operator_info(); @@ -429,8 +428,8 @@ impl Operator for Stream { let capability = capabilities.pop().unwrap(); let mut logic = constructor(capability, operator_info); move |frontiers| { - let mut input1_handle = FrontieredInputHandle::new(&mut input1, &frontiers[0]); - let mut input2_handle = FrontieredInputHandle::new(&mut input2, &frontiers[1]); + let mut input1_handle = FrontieredInputHandleCore::new(&mut input1, &frontiers[0]); + let mut input2_handle = FrontieredInputHandleCore::new(&mut input2, &frontiers[1]); let mut output_handle = output.activate(); logic(&mut input1_handle, &mut input2_handle, &mut output_handle); } @@ -439,15 +438,15 @@ impl Operator for Stream { stream } - fn binary_notify, - &mut InputHandle, - &mut OutputHandle>, + fn binary_notify, + &mut InputHandleCore, + &mut OutputHandleCore>, &mut Notificator)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract> - (&self, other: &Stream, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, mut logic: L) -> Stream { + P1: ParallelizationContractCore, + P2: ParallelizationContractCore> + (&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, init: impl IntoIterator, mut logic: L) -> StreamCore { self.binary_frontier(other, pact1, pact2, name, |capability, _info| { let mut notificator = FrontierNotificator::new(); @@ -466,16 +465,16 @@ impl Operator for Stream { } - fn binary(&self, other: &Stream, pact1: P1, pact2: P2, name: &str, constructor: B) -> Stream + fn binary(&self, other: &StreamCore, pact1: P1, pact2: P2, name: &str, constructor: B) -> StreamCore where - D2: Data, - D3: Data, + D2: Container, + D3: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut InputHandle, - &mut InputHandle, - &mut OutputHandle>)+'static, - P1: ParallelizationContract, - P2: ParallelizationContract { + L: FnMut(&mut InputHandleCore, + &mut InputHandleCore, + &mut OutputHandleCore>)+'static, + P1: ParallelizationContractCore, + P2: ParallelizationContractCore { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); let operator_info = builder.operator_info(); @@ -500,15 +499,15 @@ impl Operator for Stream { fn sink(&self, pact: P, name: &str, mut logic: L) where - L: FnMut(&mut FrontieredInputHandle)+'static, - P: ParallelizationContract { + L: FnMut(&mut FrontieredInputHandleCore)+'static, + P: ParallelizationContractCore { let mut builder = OperatorBuilder::new(name.to_owned(), self.scope()); let mut input = builder.new_input(self, pact); builder.build(|_capabilities| { move |frontiers| { - let mut input_handle = FrontieredInputHandle::new(&mut input, &frontiers[0]); + let mut input_handle = FrontieredInputHandleCore::new(&mut input, &frontiers[0]); logic(&mut input_handle); } }); @@ -556,11 +555,11 @@ impl Operator for Stream { /// .inspect(|x| println!("number: {:?}", x)); /// }); /// ``` -pub fn source(scope: &G, name: &str, constructor: B) -> Stream +pub fn source(scope: &G, name: &str, constructor: B) -> StreamCore where - D: Data, + D: Container, B: FnOnce(Capability, OperatorInfo) -> L, - L: FnMut(&mut OutputHandle>)+'static { + L: FnMut(&mut OutputHandleCore>)+'static { let mut builder = OperatorBuilder::new(name.to_owned(), scope.clone()); let operator_info = builder.operator_info(); @@ -595,12 +594,12 @@ where /// timely::example(|scope| { /// /// -/// empty(scope) //-- type required in this example -/// .inspect(|_: &()| panic!("never called")); +/// empty::<_, Vec<_>>(scope) // type required in this example +/// .inspect(|()| panic!("never called")); /// /// }); /// ``` -pub fn empty(scope: &G) -> Stream { +pub fn empty(scope: &G) -> StreamCore { source(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 11d02896d..72719c5f8 100644 --- a/timely/src/dataflow/operators/input.rs +++ b/timely/src/dataflow/operators/input.rs @@ -9,10 +9,12 @@ use crate::progress::frontier::Antichain; use crate::progress::{Operate, operate::SharedProgress, Timestamp, ChangeBatch}; use crate::progress::Source; -use crate::Data; +use crate::{Container, Data}; use crate::communication::Push; -use crate::dataflow::{Stream, ScopeParent, Scope}; -use crate::dataflow::channels::{Message, pushers::{Tee, Counter}}; +use crate::dataflow::{Stream, ScopeParent, Scope, StreamCore}; +use crate::dataflow::channels::pushers::{TeeCore, CounterCore}; +use crate::dataflow::channels::Message; + // TODO : This is an exogenous input, but it would be nice to wrap a Subgraph in something // TODO : more like a harness, with direct access to its inputs. @@ -58,6 +60,41 @@ pub trait Input : Scope { /// ``` fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream); + /// Create a new [StreamCore] and [HandleCore] through which to supply input. + /// + /// The `new_input_core` method returns a pair `(HandleCore, StreamCore)` where the [StreamCore] can be used + /// immediately for timely dataflow construction, and the `HandleCore` is later used to introduce + /// data into the timely dataflow computation. + /// + /// The `HandleCore` also provides a means to indicate + /// to timely dataflow that the input has advanced beyond certain timestamps, allowing timely + /// to issue progress notifications. + /// + /// # Examples + /// ``` + /// use timely::*; + /// use timely::dataflow::operators::{Input, Inspect}; + /// + /// // construct and execute a timely dataflow + /// timely::execute(Config::thread(), |worker| { + /// + /// // add an input and base computation off of it + /// let mut input = worker.dataflow(|scope| { + /// let (input, stream) = scope.new_input_core::>(); + /// stream.inspect(|x| println!("hello {:?}", x)); + /// input + /// }); + /// + /// // introduce input, advance computation + /// for round in 0..10 { + /// input.send(round); + /// input.advance_to(round + 1); + /// worker.step(); + /// } + /// }); + /// ``` + fn new_input_core(&mut self) -> (HandleCore<::Timestamp, D>, StreamCore); + /// Create a new stream from a supplied interactive handle. /// /// This method creates a new timely stream whose data are supplied interactively through the `handle` @@ -89,20 +126,59 @@ pub trait Input : Scope { /// }); /// ``` fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream; + + /// Create a new stream from a supplied interactive handle. + /// + /// This method creates a new timely stream whose data are supplied interactively through the `handle` + /// argument. Each handle may be used multiple times (or not at all), and will clone data as appropriate + /// if it as attached to more than one stream. + /// + /// # Examples + /// ``` + /// use timely::*; + /// use timely::dataflow::operators::{Input, Inspect}; + /// use timely::dataflow::operators::input::Handle; + /// + /// // construct and execute a timely dataflow + /// timely::execute(Config::thread(), |worker| { + /// + /// // add an input and base computation off of it + /// let mut input = Handle::new(); + /// worker.dataflow(|scope| { + /// scope.input_from_core(&mut input) + /// .inspect(|x| println!("hello {:?}", x)); + /// }); + /// + /// // introduce input, advance computation + /// for round in 0..10 { + /// input.send(round); + /// input.advance_to(round + 1); + /// worker.step(); + /// } + /// }); + /// ``` + fn input_from_core(&mut self, handle: &mut HandleCore<::Timestamp, D>) -> StreamCore; } use crate::order::TotalOrder; impl Input for G where ::Timestamp: TotalOrder { fn new_input(&mut self) -> (Handle<::Timestamp, D>, Stream) { - let mut handle = Handle::new(); - let stream = self.input_from(&mut handle); - (handle, stream) + self.new_input_core() } fn input_from(&mut self, handle: &mut Handle<::Timestamp, D>) -> Stream { + self.input_from_core(handle) + } + + fn new_input_core(&mut self) -> (HandleCore<::Timestamp, D>, StreamCore) { + let mut handle = HandleCore::new(); + let stream = self.input_from_core(&mut handle); + (handle, stream) + } - let (output, registrar) = Tee::<::Timestamp, D>::new(); - let counter = Counter::new(output); + fn input_from_core(&mut self, handle: &mut HandleCore<::Timestamp, D>) -> StreamCore { + let (output, registrar) = TeeCore::<::Timestamp, D>::new(); + let counter = CounterCore::new(output); let produced = counter.produced().clone(); let index = self.allocate_operator_index(); @@ -126,7 +202,7 @@ impl Input for G where ::Timestamp: TotalOrder { copies, }), index); - Stream::new(Source::new(index, 0), registrar, self.clone()) + StreamCore::new(Source::new(index, 0), registrar, self.clone()) } } @@ -170,17 +246,19 @@ impl Operate for Operator { /// A handle to an input `Stream`, used to introduce data to a timely dataflow computation. #[derive(Debug)] -pub struct Handle { +pub struct HandleCore { activate: Vec, progress: Vec>>>, - pushers: Vec>>, - buffer1: Vec, - buffer2: Vec, + pushers: Vec>>, + buffer1: C, + buffer2: C, now_at: T, } -impl Handle { +/// A handle specialized to vector-based containers. +pub type Handle = HandleCore>; +impl HandleCore { /// Allocates a new input handle, from which one can create timely streams. /// /// # Examples @@ -208,12 +286,12 @@ impl Handle { /// }); /// ``` pub fn new() -> Self { - Handle { + Self { activate: Vec::new(), progress: Vec::new(), pushers: Vec::new(), - buffer1: Vec::with_capacity(Message::::default_length()), - buffer2: Vec::with_capacity(Message::::default_length()), + buffer1: Default::default(), + buffer2: Default::default(), now_at: T::minimum(), } } @@ -244,18 +322,18 @@ impl Handle { /// } /// }); /// ``` - pub fn to_stream(&mut self, scope: &mut G) -> Stream + pub fn to_stream(&mut self, scope: &mut G) -> StreamCore where T: TotalOrder, G: ScopeParent, { - scope.input_from(self) + scope.input_from_core(self) } fn register( &mut self, - pusher: Counter>, - progress: Rc>> + pusher: CounterCore>, + progress: Rc>>, ) { // flush current contents, so new registrant does not see existing data. if !self.buffer1.is_empty() { self.flush(); } @@ -274,7 +352,7 @@ impl Handle { fn flush(&mut self) { for index in 0 .. self.pushers.len() { if index < self.pushers.len() - 1 { - self.buffer2.extend_from_slice(&self.buffer1[..]); + self.buffer2.clone_from(&self.buffer1); Message::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); debug_assert!(self.buffer2.is_empty()); } @@ -301,20 +379,35 @@ impl Handle { } } - #[inline] - /// Sends one record into the corresponding timely dataflow `Stream`, at the current epoch. - pub fn send(&mut self, data: D) { - // assert!(self.buffer1.capacity() == Message::::default_length()); - self.buffer1.push(data); - if self.buffer1.len() == self.buffer1.capacity() { - self.flush(); - } - } - - /// Sends a batch of records into the corresponding timely dataflow `Stream`, at the current epoch. + /// Sends a batch of records into the corresponding timely dataflow [StreamCore], at the current epoch. /// /// This method flushes single elements previously sent with `send`, to keep the insertion order. - pub fn send_batch(&mut self, buffer: &mut Vec) { + /// + /// # Examples + /// ``` + /// use timely::*; + /// use timely::dataflow::operators::{Input, InspectCore}; + /// use timely::dataflow::operators::input::HandleCore; + /// + /// // construct and execute a timely dataflow + /// timely::execute(Config::thread(), |worker| { + /// + /// // add an input and base computation off of it + /// let mut input = HandleCore::new(); + /// worker.dataflow(|scope| { + /// scope.input_from_core(&mut input) + /// .inspect_container(|x| println!("hello {:?}", x)); + /// }); + /// + /// // introduce input, advance computation + /// for round in 0..10 { + /// input.send_batch(&mut vec![format!("{}", round)]); + /// input.advance_to(round + 1); + /// worker.step(); + /// } + /// }); + /// ``` + pub fn send_batch(&mut self, buffer: &mut D) { if !buffer.is_empty() { // flush buffered elements to ensure local fifo. @@ -323,7 +416,7 @@ impl Handle { // push buffer (or clone of buffer) at each destination. for index in 0 .. self.pushers.len() { if index < self.pushers.len() - 1 { - self.buffer2.extend_from_slice(&buffer[..]); + self.buffer2.clone_from(&buffer); Message::push_at(&mut self.buffer2, self.now_at.clone(), &mut self.pushers[index]); assert!(self.buffer2.is_empty()); } @@ -370,13 +463,50 @@ impl Handle { } } +impl Handle { + #[inline] + /// Sends one record into the corresponding timely dataflow `Stream`, at the current epoch. + /// + /// # Examples + /// ``` + /// use timely::*; + /// use timely::dataflow::operators::{Input, Inspect}; + /// use timely::dataflow::operators::input::Handle; + /// + /// // construct and execute a timely dataflow + /// timely::execute(Config::thread(), |worker| { + /// + /// // add an input and base computation off of it + /// let mut input = Handle::new(); + /// worker.dataflow(|scope| { + /// scope.input_from(&mut input) + /// .inspect(|x| println!("hello {:?}", x)); + /// }); + /// + /// // introduce input, advance computation + /// for round in 0..10 { + /// input.send(round); + /// input.advance_to(round + 1); + /// worker.step(); + /// } + /// }); + /// ``` + pub fn send(&mut self, data: D) { + // assert!(self.buffer1.capacity() == Message::::default_length()); + self.buffer1.push(data); + if self.buffer1.len() == self.buffer1.capacity() { + self.flush(); + } + } +} + impl Default for Handle { fn default() -> Self { Self::new() } } -impl Drop for Handle { +impl Drop for HandleCore { fn drop(&mut self) { self.close_epoch(); } diff --git a/timely/src/dataflow/operators/inspect.rs b/timely/src/dataflow/operators/inspect.rs index a8f09f787..6e26856d1 100644 --- a/timely/src/dataflow/operators/inspect.rs +++ b/timely/src/dataflow/operators/inspect.rs @@ -1,12 +1,15 @@ //! Extension trait and implementation for observing and action on streamed data. +use std::rc::Rc; +use timely_container::columnation::{Columnation, TimelyStack}; +use crate::Container; use crate::Data; use crate::dataflow::channels::pact::Pipeline; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::operators::generic::Operator; /// Methods to inspect records and batches of records on a stream. -pub trait Inspect { +pub trait Inspect: InspectCore + Sized { /// Runs a supplied closure on each observed data element. /// /// # Examples @@ -18,7 +21,7 @@ pub trait Inspect { /// .inspect(|x| println!("seen: {:?}", x)); /// }); /// ``` - fn inspect(&self, mut func: impl FnMut(&D)+'static) -> Stream { + fn inspect(&self, mut func: impl FnMut(&C::Item)+'static) -> Self { self.inspect_batch(move |_, data| { for datum in data.iter() { func(datum); } }) @@ -35,7 +38,7 @@ pub trait Inspect { /// .inspect_time(|t, x| println!("seen at: {:?}\t{:?}", t, x)); /// }); /// ``` - fn inspect_time(&self, mut func: impl FnMut(&G::Timestamp, &D)+'static) -> Stream { + fn inspect_time(&self, mut func: impl FnMut(&G::Timestamp, &C::Item)+'static) -> Self { self.inspect_batch(move |time, data| { for datum in data.iter() { func(&time, &datum); @@ -54,7 +57,7 @@ pub trait Inspect { /// .inspect_batch(|t,xs| println!("seen at: {:?}\t{:?} records", t, xs.len())); /// }); /// ``` - fn inspect_batch(&self, mut func: impl FnMut(&G::Timestamp, &[D])+'static) -> Stream { + fn inspect_batch(&self, mut func: impl FnMut(&G::Timestamp, &[C::Item])+'static) -> Self { self.inspect_core(move |event| { if let Ok((time, data)) = event { func(time, data); @@ -81,17 +84,61 @@ pub trait Inspect { /// }); /// }); /// ``` - fn inspect_core(&self, func: F) -> Stream where F: FnMut(Result<(&G::Timestamp, &[D]), &[G::Timestamp]>)+'static; + fn inspect_core(&self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &[C::Item]), &[G::Timestamp]>)+'static; } -impl Inspect for Stream { +impl Inspect> for StreamCore> { + fn inspect_core(&self, mut func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &[D]), &[G::Timestamp]>) + 'static { + self.inspect_container(move |r| func(r.map(|(t, c)| (t, &c[..])))) + } +} + +impl Inspect> for StreamCore> { + fn inspect_core(&self, mut func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &[D]), &[G::Timestamp]>) + 'static { + self.inspect_container(move |r| func(r.map(|(t, c)| (t, &c[..])))) + } +} + +impl Inspect> for StreamCore> + where C: AsRef<[C::Item]> +{ + fn inspect_core(&self, mut func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &[C::Item]), &[G::Timestamp]>) + 'static { + self.inspect_container(move |r| func(r.map(|(t, c)| (t, c.as_ref().as_ref())))) + } +} + +/// Inspect containers +pub trait InspectCore { + /// Runs a supplied closure on each observed container, and each frontier advancement. + /// + /// Rust's `Result` type is used to distinguish the events, with `Ok` for time and data, + /// and `Err` for frontiers. Frontiers are only presented when they change. + /// + /// # Examples + /// ``` + /// use timely::dataflow::operators::{ToStream, Map, InspectCore}; + /// + /// timely::example(|scope| { + /// (0..10).to_stream(scope) + /// .inspect_container(|event| { + /// match event { + /// Ok((time, data)) => println!("seen at: {:?}\t{:?} records", time, data.len()), + /// Err(frontier) => println!("frontier advanced to {:?}", frontier), + /// } + /// }); + /// }); + /// ``` + fn inspect_container(&self, func: F) -> StreamCore where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static; +} + +impl InspectCore for StreamCore { - fn inspect_core(&self, mut func: F) -> Stream - where F: FnMut(Result<(&G::Timestamp, &[D]), &[G::Timestamp]>)+'static + fn inspect_container(&self, mut func: F) -> StreamCore + where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static { use crate::progress::timestamp::Timestamp; - let mut vector = Vec::new(); let mut frontier = crate::progress::Antichain::from_elem(G::Timestamp::minimum()); + let mut vector = Default::default(); self.unary_frontier(Pipeline, "InspectBatch", move |_,_| move |input, output| { if input.frontier.frontier() != frontier.borrow() { frontier.clear(); @@ -100,8 +147,8 @@ impl Inspect for Stream { } input.for_each(|time, data| { data.swap(&mut vector); - func(Ok((&time, &vector[..]))); - output.session(&time).give_vec(&mut vector); + func(Ok((&time, &vector))); + output.session(&time).give_container(&mut vector); }); }) } diff --git a/timely/src/dataflow/operators/mod.rs b/timely/src/dataflow/operators/mod.rs index 7b418df46..1eec15cba 100644 --- a/timely/src/dataflow/operators/mod.rs +++ b/timely/src/dataflow/operators/mod.rs @@ -9,20 +9,19 @@ //! Most of the operators in this module are defined using these two general operators. pub use self::enterleave::{Enter, EnterAt, Leave}; -// pub use self::queue::*; pub use self::input::Input; -pub use self::unordered_input::UnorderedInput; +pub use self::unordered_input::{UnorderedInput, UnorderedInputCore}; pub use self::feedback::{Feedback, LoopVariable, ConnectLoop}; pub use self::concat::{Concat, Concatenate}; pub use self::partition::Partition; pub use self::map::Map; -pub use self::inspect::Inspect; +pub use self::inspect::{Inspect, InspectCore}; pub use self::filter::Filter; pub use self::delay::Delay; pub use self::exchange::Exchange; pub use self::broadcast::Broadcast; pub use self::probe::Probe; -pub use self::to_stream::{ToStream, ToStreamAsync, Event}; +pub use self::to_stream::{ToStream, ToStreamCore}; pub use self::capture::Capture; pub use self::branch::{Branch, BranchWhen}; pub use self::ok_err::OkErr; @@ -52,6 +51,7 @@ pub mod to_stream; pub mod capture; pub mod branch; pub mod ok_err; +pub mod rc; pub mod result; pub mod aggregation; @@ -62,4 +62,4 @@ pub mod count; // keep "mint" module-private mod capability; -pub use self::capability::{ActivateCapability, Capability, CapabilityRef, CapabilitySet}; +pub use self::capability::{ActivateCapability, Capability, InputCapability, CapabilitySet, DowngradeError}; diff --git a/timely/src/dataflow/operators/probe.rs b/timely/src/dataflow/operators/probe.rs index 124be5972..ad990cb79 100644 --- a/timely/src/dataflow/operators/probe.rs +++ b/timely/src/dataflow/operators/probe.rs @@ -5,18 +5,18 @@ use std::cell::RefCell; use crate::progress::Timestamp; use crate::progress::frontier::{AntichainRef, MutableAntichain}; -use crate::dataflow::channels::pushers::Counter as PushCounter; -use crate::dataflow::channels::pushers::buffer::Buffer as PushBuffer; +use crate::dataflow::channels::pushers::CounterCore as PushCounter; +use crate::dataflow::channels::pushers::buffer::BufferCore 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::Data; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{StreamCore, Scope}; +use crate::Container; /// Monitors progress at a `Stream`. -pub trait Probe { +pub trait Probe { /// Constructs a progress probe which indicates which timestamps have elapsed at the operator. /// /// # Examples @@ -76,10 +76,10 @@ pub trait Probe { /// } /// }).unwrap(); /// ``` - fn probe_with(&self, handle: &mut Handle) -> Stream; + fn probe_with(&self, handle: &Handle) -> StreamCore; } -impl Probe for Stream { +impl Probe for StreamCore { fn probe(&self) -> Handle { // the frontier is shared state; scope updates, handle reads. @@ -87,24 +87,26 @@ impl Probe for Stream { self.probe_with(&mut handle); handle } - fn probe_with(&self, handle: &mut Handle) -> Stream { + 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)); let (tee, stream) = builder.new_output(); let mut output = PushBuffer::new(PushCounter::new(tee)); - let shared_frontier = handle.frontier.clone(); + let shared_frontier = Rc::downgrade(&handle.frontier); let mut started = false; - let mut vector = Vec::new(); + let mut vector = Default::default(); builder.build( move |progress| { // surface all frontier changes to the shared frontier. - let mut borrow = shared_frontier.borrow_mut(); - borrow.update_iter(progress.frontiers[0].drain()); + if let Some(shared_frontier) = shared_frontier.upgrade() { + let mut borrow = shared_frontier.borrow_mut(); + borrow.update_iter(progress.frontiers[0].drain()); + } if !started { // discard initial capability. @@ -120,7 +122,7 @@ impl Probe for Stream { RefOrMut::Mut(reference) => (&reference.time, RefOrMut::Mut(&mut reference.data)), }; data.swap(&mut vector); - output.session(time).give_vec(&mut vector); + output.session(time).give_container(&mut vector); } output.cease(); diff --git a/timely/src/dataflow/operators/rc.rs b/timely/src/dataflow/operators/rc.rs new file mode 100644 index 000000000..eaae55093 --- /dev/null +++ b/timely/src/dataflow/operators/rc.rs @@ -0,0 +1,82 @@ +//! Shared containers + +use crate::dataflow::channels::pact::Pipeline; +use crate::dataflow::operators::Operator; +use crate::dataflow::{Scope, StreamCore}; +use crate::Container; +use std::rc::Rc; + +/// Convert a stream into a stream of shared containers +pub trait SharedStream { + /// Convert a stream into a stream of shared data + /// + /// # Examples + /// ``` + /// use timely::dataflow::operators::{ToStream, Inspect}; + /// use timely::dataflow::operators::rc::SharedStream; + /// + /// timely::example(|scope| { + /// (0..10).to_stream(scope) + /// .shared() + /// .inspect(|x| println!("seen: {:?}", x)); + /// }); + /// ``` + fn shared(&self) -> StreamCore>; +} + +impl SharedStream for StreamCore { + fn shared(&self) -> StreamCore> { + let mut container = Default::default(); + self.unary(Pipeline, "Shared", move |_, _| { + move |input, output| { + input.for_each(|time, data| { + data.swap(&mut container); + output + .session(&time) + .give_container(&mut Rc::new(std::mem::take(&mut container))); + }); + } + }) + } +} + +#[cfg(test)] +mod test { + use crate::dataflow::channels::pact::Pipeline; + use crate::dataflow::operators::capture::Extract; + use crate::dataflow::operators::rc::SharedStream; + use crate::dataflow::operators::{Capture, Concatenate, Operator, ToStream}; + + #[test] + fn test_shared() { + let output = crate::example(|scope| { + let shared = vec![Ok(0), Err(())].to_stream(scope).shared(); + scope + .concatenate([ + shared.unary(Pipeline, "read shared 1", |_, _| { + let mut container = Default::default(); + move |input, output| { + input.for_each(|time, data| { + data.swap(&mut container); + output.session(&time).give(container.as_ptr() as usize); + }); + } + }), + shared.unary(Pipeline, "read shared 2", |_, _| { + let mut container = Default::default(); + move |input, output| { + input.for_each(|time, data| { + data.swap(&mut container); + output.session(&time).give(container.as_ptr() as usize); + }); + } + }), + ]) + .capture() + }); + let output = &mut output.extract()[0].1; + output.sort(); + output.dedup(); + assert_eq!(output.len(), 1); + } +} diff --git a/timely/src/dataflow/operators/reclock.rs b/timely/src/dataflow/operators/reclock.rs index 65d4f89e2..b656e0aaf 100644 --- a/timely/src/dataflow/operators/reclock.rs +++ b/timely/src/dataflow/operators/reclock.rs @@ -1,13 +1,13 @@ //! Extension methods for `Stream` based on record-by-record transformation. -use crate::Data; +use crate::Container; use crate::order::PartialOrder; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{Scope, StreamCore}; use crate::dataflow::channels::pact::Pipeline; use crate::dataflow::operators::generic::operator::Operator; /// Extension trait for reclocking a stream. -pub trait Reclock { +pub trait Reclock { /// Delays records until an input is observed on the `clock` input. /// /// The source stream is buffered until a record is seen on the clock input, @@ -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: &Stream) -> Stream; + fn reclock>(&self, clock: &StreamCore) -> Self; } -impl Reclock for Stream { - fn reclock(&self, clock: &Stream) -> Stream { +impl Reclock for StreamCore { + fn reclock>(&self, clock: &StreamCore) -> StreamCore { let mut stash = vec![]; @@ -57,7 +57,7 @@ impl Reclock for Stream { // stash each data input with its timestamp. input1.for_each(|cap, data| { - stash.push((cap.time().clone(), data.replace(Vec::new()))); + stash.push((cap.time().clone(), data.replace(Default::default()))); }); // request notification at time, to flush stash. @@ -70,7 +70,7 @@ impl Reclock for Stream { let mut session = output.session(&cap); for &mut (ref t, ref mut data) in &mut stash { if t.less_equal(cap.time()) { - session.give_vec(data); + session.give_container(data); } } stash.retain(|x| !x.0.less_equal(cap.time())); diff --git a/timely/src/dataflow/operators/to_stream.rs b/timely/src/dataflow/operators/to_stream.rs index 5a1709620..555a540b0 100644 --- a/timely/src/dataflow/operators/to_stream.rs +++ b/timely/src/dataflow/operators/to_stream.rs @@ -1,15 +1,10 @@ //! Conversion to the `Stream` type from iterators. -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use crate::dataflow::channels::Message; -use crate::dataflow::operators::generic::operator::source; -use crate::dataflow::operators::CapabilitySet; -use crate::dataflow::{Scope, Stream}; +use crate::Container; use crate::progress::Timestamp; use crate::Data; +use crate::dataflow::operators::generic::operator::source; +use crate::dataflow::{StreamCore, Stream, Scope}; /// Converts to a timely `Stream`. pub trait ToStream { @@ -48,7 +43,8 @@ impl ToStream for I where I:: if let Some(element) = iterator.next() { let mut session = output.session(capability.as_ref().unwrap()); session.give(element); - for element in iterator.by_ref().take((256 * Message::::default_length()) - 1) { + let n = 256 * crate::container::buffer::default_capacity::(); + for element in iterator.by_ref().take(n - 1) { session.give(element); } activator.activate(); @@ -61,77 +57,51 @@ impl ToStream for I where I:: } } -/// Data and progress events of the native stream. -pub enum Event { - /// Indicates that timestamps have advanced to frontier F - Progress(F), - /// Indicates that event D happened at time T - Message(F::Item, D), -} - -/// Converts to a timely `Stream`. -pub trait ToStreamAsync { - /// Converts a [native `Stream`](futures_util::stream::Stream) of [`Event`s](Event) into a [timely - /// `Stream`](crate::dataflow::Stream). +/// Converts to a timely [StreamCore]. +pub trait ToStreamCore { + /// Converts to a timely [StreamCore]. /// /// # Examples /// /// ``` - /// use futures_util::stream; - /// - /// use timely::dataflow::operators::{Capture, Event, ToStream, ToStreamAsync}; + /// use timely::dataflow::operators::{ToStreamCore, Capture}; /// use timely::dataflow::operators::capture::Extract; /// - /// let native_stream = stream::iter(vec![ - /// Event::Message(0, 0), - /// Event::Message(0, 1), - /// Event::Message(0, 2), - /// Event::Progress(Some(0)), - /// ]); - /// /// let (data1, data2) = timely::example(|scope| { - /// let data1 = native_stream.to_stream(scope).capture(); - /// let data2 = vec![0,1,2].to_stream(scope).capture(); - /// + /// let data1 = Some((0..3).collect::>()).to_stream_core(scope).capture(); + /// let data2 = Some(vec![0,1,2]).to_stream_core(scope).capture(); /// (data1, data2) /// }); /// /// assert_eq!(data1.extract(), data2.extract()); /// ``` - fn to_stream>(self, scope: &S) -> Stream; + fn to_stream_core>(self, scope: &mut S) -> StreamCore; } -impl ToStreamAsync for I -where - D: Data, - T: Timestamp, - F: IntoIterator, - I: futures_util::stream::Stream> + Unpin + 'static, -{ - fn to_stream>(mut self, scope: &S) -> Stream { - source(scope, "ToStreamAsync", move |capability, info| { - let activator = Arc::new(scope.sync_activator_for(&info.address[..])); +impl ToStreamCore for I where I::Item: Container { + fn to_stream_core>(self, scope: &mut S) -> StreamCore { - let mut cap_set = CapabilitySet::from_elem(capability); + source(scope, "ToStreamCore", |capability, info| { + + // Acquire an activator, so that the operator can rescheduled itself. + let activator = scope.activator_for(&info.address[..]); + + let mut iterator = self.into_iter().fuse(); + let mut capability = Some(capability); move |output| { - let waker = futures_util::task::waker_ref(&activator); - let mut context = Context::from_waker(&waker); - // Consume all the ready items of the source_stream and issue them to the operator - while let Poll::Ready(item) = Pin::new(&mut self).poll_next(&mut context) { - match item { - Some(Event::Progress(time)) => { - cap_set.downgrade(time); - } - Some(Event::Message(time, data)) => { - output.session(&cap_set.delayed(&time)).give(data); - } - None => { - cap_set.downgrade(&[]); - break; - } + if let Some(mut element) = iterator.next() { + let mut session = output.session(capability.as_ref().unwrap()); + session.give_container(&mut element); + let n = 256; + for mut element in iterator.by_ref().take(n - 1) { + session.give_container(&mut element); } + activator.activate(); + } + else { + capability = None; } } }) diff --git a/timely/src/dataflow/operators/unordered_input.rs b/timely/src/dataflow/operators/unordered_input.rs index 6dc163bbf..c7e600234 100644 --- a/timely/src/dataflow/operators/unordered_input.rs +++ b/timely/src/dataflow/operators/unordered_input.rs @@ -2,6 +2,7 @@ use std::rc::Rc; use std::cell::RefCell; +use crate::Container; use crate::scheduling::{Schedule, ActivateOnDrop}; @@ -11,12 +12,12 @@ use crate::progress::Source; use crate::progress::ChangeBatch; use crate::Data; -use crate::dataflow::channels::pushers::{Tee, Counter as PushCounter}; -use crate::dataflow::channels::pushers::buffer::{Buffer as PushBuffer, AutoflushSession}; +use crate::dataflow::channels::pushers::{CounterCore as PushCounter, TeeCore}; +use crate::dataflow::channels::pushers::buffer::{BufferCore as PushBuffer, AutoflushSessionCore}; use crate::dataflow::operators::{ActivateCapability, Capability}; -use crate::dataflow::{Stream, Scope}; +use crate::dataflow::{Stream, Scope, StreamCore}; /// Create a new `Stream` and `Handle` through which to supply input. pub trait UnorderedInput { @@ -79,8 +80,76 @@ pub trait UnorderedInput { impl UnorderedInput for G { fn new_unordered_input(&mut self) -> ((UnorderedHandle, ActivateCapability), Stream) { + self.new_unordered_input_core() + } +} + +/// An unordered handle specialized to vectors. +pub type UnorderedHandle = UnorderedHandleCore>; + +/// Create a new `Stream` and `Handle` through which to supply input. +pub trait UnorderedInputCore { + /// Create a new capability-based [StreamCore] and [UnorderedHandleCore] through which to supply input. This + /// input supports multiple open epochs (timestamps) at the same time. + /// + /// The `new_unordered_input_core` method returns `((HandleCore, Capability), StreamCore)` where the `StreamCore` can be used + /// immediately for timely dataflow construction, `HandleCore` and `Capability` are later used to introduce + /// data into the timely dataflow computation. + /// + /// The `Capability` returned is for the default value of the timestamp type in use. The + /// capability can be dropped to inform the system that the input has advanced beyond the + /// capability's timestamp. To retain the ability to send, a new capability at a later timestamp + /// should be obtained first, via the `delayed` function for `Capability`. + /// + /// To communicate the end-of-input drop all available capabilities. + /// + /// # Examples + /// + /// ``` + /// use std::sync::{Arc, Mutex}; + /// + /// use timely::*; + /// use timely::dataflow::operators::*; + /// use timely::dataflow::operators::capture::Extract; + /// use timely::dataflow::Stream; + /// + /// // get send and recv endpoints, wrap send to share + /// let (send, recv) = ::std::sync::mpsc::channel(); + /// let send = Arc::new(Mutex::new(send)); + /// + /// timely::execute(Config::thread(), move |worker| { + /// + /// // this is only to validate the output. + /// let send = send.lock().unwrap().clone(); + /// + /// // create and capture the unordered input. + /// let (mut input, mut cap) = worker.dataflow::(|scope| { + /// let (input, stream) = scope.new_unordered_input_core(); + /// stream.capture_into(send); + /// input + /// }); + /// + /// // feed values 0..10 at times 0..10. + /// for round in 0..10 { + /// input.session(cap.clone()).give(round); + /// cap = cap.delayed(&(round + 1)); + /// worker.step(); + /// } + /// }).unwrap(); + /// + /// let extract = recv.extract(); + /// for i in 0..10 { + /// assert_eq!(extract[i], (i, vec![i])); + /// } + /// ``` + fn new_unordered_input_core(&mut self) -> ((UnorderedHandleCore, ActivateCapability), StreamCore); +} + + +impl UnorderedInputCore for G { + fn new_unordered_input_core(&mut self) -> ((UnorderedHandleCore, ActivateCapability), StreamCore) { - let (output, registrar) = Tee::::new(); + let (output, registrar) = TeeCore::::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()); @@ -94,7 +163,7 @@ impl UnorderedInput for G { let cap = ActivateCapability::new(cap, &address, self.activations()); - let helper = UnorderedHandle::new(counter); + let helper = UnorderedHandleCore::new(counter); self.add_operator_with_index(Box::new(UnorderedOperator { name: "UnorderedInput".to_owned(), @@ -105,7 +174,7 @@ impl UnorderedInput for G { peers, }), index); - ((helper, cap), Stream::new(Source::new(index, 0), registrar, self.clone())) + ((helper, cap), StreamCore::new(Source::new(index, 0), registrar, self.clone())) } } @@ -144,21 +213,21 @@ impl Operate for UnorderedOperator { fn notify_me(&self) -> bool { false } } -/// A handle to an input `Stream`, used to introduce data to a timely dataflow computation. +/// A handle to an input [StreamCore], used to introduce data to a timely dataflow computation. #[derive(Debug)] -pub struct UnorderedHandle { - buffer: PushBuffer>>, +pub struct UnorderedHandleCore { + buffer: PushBuffer>>, } -impl UnorderedHandle { - fn new(pusher: PushCounter>) -> UnorderedHandle { - UnorderedHandle { +impl UnorderedHandleCore { + fn new(pusher: PushCounter>) -> UnorderedHandleCore { + UnorderedHandleCore { buffer: PushBuffer::new(pusher), } } /// Allocates a new automatically flushing session based on the supplied capability. - pub fn session<'b>(&'b mut self, cap: ActivateCapability) -> ActivateOnDrop>>> { + pub fn session<'b>(&'b mut self, cap: ActivateCapability) -> ActivateOnDrop>>> { ActivateOnDrop::new(self.buffer.autoflush_session(cap.capability.clone()), cap.address.clone(), cap.activations.clone()) } } diff --git a/timely/src/dataflow/stream.rs b/timely/src/dataflow/stream.rs index 8a3bccfbd..aa5e48601 100644 --- a/timely/src/dataflow/stream.rs +++ b/timely/src/dataflow/stream.rs @@ -9,17 +9,18 @@ use crate::progress::{Source, Target}; use crate::communication::Push; use crate::dataflow::Scope; use crate::dataflow::channels::pushers::tee::TeeHelper; -use crate::dataflow::channels::Bundle; +use crate::dataflow::channels::BundleCore; use std::fmt::{self, Debug}; +use crate::Container; // use dataflow::scopes::root::loggers::CHANNELS_Q; -/// Abstraction of a stream of `D: Data` records timestamped with `S::Timestamp`. +/// Abstraction of a stream of `D: Container` records timestamped with `S::Timestamp`. /// /// Internally `Stream` maintains a list of data recipients who should be presented with data /// produced by the source of the stream. #[derive(Clone)] -pub struct Stream { +pub struct StreamCore { /// The progress identifier of the stream's data source. name: Source, /// The `Scope` containing the stream. @@ -28,12 +29,15 @@ pub struct Stream { ports: TeeHelper, } -impl Stream { +/// A stream batching data in vectors. +pub type Stream = 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 { @@ -48,7 +52,7 @@ impl Stream { } /// Allocates a `Stream` from a supplied `Source` name and rendezvous point. pub fn new(source: Source, output: TeeHelper, scope: S) -> Self { - Stream { name: source, ports: output, scope } + Self { name: source, ports: output, scope } } /// The name of the stream's source operator. pub fn name(&self) -> &Source { &self.name } @@ -56,7 +60,7 @@ impl Stream { pub fn scope(&self) -> S { self.scope.clone() } } -impl Debug for Stream +impl Debug for StreamCore where S: Scope, { diff --git a/timely/src/execute.rs b/timely/src/execute.rs index 96714e737..3564d3592 100644 --- a/timely/src/execute.rs +++ b/timely/src/execute.rs @@ -14,7 +14,7 @@ pub struct Config { } impl Config { - /// Installs options into a [`getopts::Options`] struct that correspond + /// Installs options into a [getopts_dep::Options] struct that correspond /// to the parameters in the configuration. /// /// It is the caller's responsibility to ensure that the installed options @@ -32,8 +32,8 @@ impl Config { /// Instantiates a configuration based upon the parsed options in `matches`. /// /// The `matches` object must have been constructed from a - /// [`getopts::Options`] which contained at least the options installed by - /// [`Self::install_options`]. + /// [getopts_dep::Options] which contained at least the options installed by + /// [Self::install_options]. /// /// This method is only available if the `getopts` feature is enabled, which /// it is by default. @@ -155,7 +155,9 @@ where let alloc = crate::communication::allocator::thread::Thread::new(); let mut worker = crate::worker::Worker::new(WorkerConfig::default(), alloc); let result = func(&mut worker); - while worker.step_or_park(None) { } + while worker.has_dataflows() { + worker.step_or_park(None); + } result } @@ -233,12 +235,12 @@ where use ::std::net::TcpStream; use crate::logging::BatchLogger; - use crate::dataflow::operators::capture::EventWriter; + use crate::dataflow::operators::capture::EventWriterCore; eprintln!("enabled COMM logging to {}", addr); if let Ok(stream) = TcpStream::connect(&addr) { - let writer = EventWriter::new(stream); + let writer = EventWriterCore::new(stream); let mut logger = BatchLogger::new(writer); result = Some(crate::logging_core::Logger::new( ::std::time::Instant::now(), @@ -267,10 +269,10 @@ where use ::std::net::TcpStream; use crate::logging::{BatchLogger, TimelyEvent}; - use crate::dataflow::operators::capture::EventWriter; + use crate::dataflow::operators::capture::EventWriterCore; if let Ok(stream) = TcpStream::connect(&addr) { - let writer = EventWriter::new(stream); + let writer = EventWriterCore::new(stream); let mut logger = BatchLogger::new(writer); worker.log_register() .insert::("timely", move |time, data| @@ -283,7 +285,9 @@ where } let result = func(&mut worker); - while worker.step_or_park(None) { } + while worker.has_dataflows() { + worker.step_or_park(None); + } result }) } @@ -382,7 +386,9 @@ where initialize_from(builders, others, move |allocator| { let mut worker = Worker::new(worker_config.clone(), allocator); let result = func(&mut worker); - while worker.step_or_park(None) { } + while worker.has_dataflows() { + worker.step_or_park(None); + } result }) } diff --git a/timely/src/lib.rs b/timely/src/lib.rs index 2241ae39f..96d7c7d99 100644 --- a/timely/src/lib.rs +++ b/timely/src/lib.rs @@ -76,6 +76,12 @@ pub use timely_communication::Config as CommunicationConfig; pub use worker::Config as WorkerConfig; pub use execute::Config as Config; +pub use timely_container::Container; +/// Re-export of the `timely_container` crate. +pub mod container { + pub use timely_container::*; +} + /// Re-export of the `timely_communication` crate. pub mod communication { pub use timely_communication::*; diff --git a/timely/src/logging.rs b/timely/src/logging.rs index 4a59e057f..ca9868fbe 100644 --- a/timely/src/logging.rs +++ b/timely/src/logging.rs @@ -76,7 +76,7 @@ pub struct ChannelsEvent { pub trait ProgressEventTimestamp: std::fmt::Debug + std::any::Any { /// Upcasts this `ProgressEventTimestamp` to `Any`. /// - /// NOTE: This is required until https://github.com/rust-lang/rfcs/issues/2765 is fixed + /// NOTE: This is required until is fixed /// /// # Example /// ```rust diff --git a/timely/src/order.rs b/timely/src/order.rs index 0564c0a62..45cdda7d7 100644 --- a/timely/src/order.rs +++ b/timely/src/order.rs @@ -25,6 +25,15 @@ pub trait PartialOrder : Eq { /// and other sanity-maintaining operations. pub trait TotalOrder : PartialOrder { } +/// A type that does not affect total orderedness. +/// +/// This trait is not useful, but must be made public and documented or else Rust +/// complains about its existence in the constraints on the implementation of +/// public traits for public types. +pub trait Empty : PartialOrder { } + +impl Empty for () { } + macro_rules! implement_partial { ($($index_type:ty,)*) => ( $( @@ -47,93 +56,189 @@ macro_rules! implement_total { implement_partial!(u8, u16, u32, u64, u128, usize, i8, i16, i32, i64, i128, isize, (), ::std::time::Duration,); implement_total!(u8, u16, u32, u64, u128, usize, i8, i16, i32, i64, i128, isize, (), ::std::time::Duration,); +pub use product::Product; +/// A pair of timestamps, partially ordered by the product order. +mod product { + use std::fmt::{Formatter, Error, Debug}; + + use crate::container::columnation::{Columnation, Region}; + use crate::order::{Empty, TotalOrder}; + use crate::progress::Timestamp; + use crate::progress::timestamp::PathSummary; + use crate::progress::timestamp::Refines; + + /// A nested pair of timestamps, one outer and one inner. + /// + /// We use `Product` rather than `(TOuter, TInner)` so that we can derive our own `PartialOrder`, + /// because Rust just uses the lexicographic total order. + #[derive(Abomonation, Copy, Clone, Hash, Eq, PartialEq, Default, Ord, PartialOrd, Serialize, Deserialize)] + pub struct Product { + /// Outer timestamp. + pub outer: TOuter, + /// Inner timestamp. + pub inner: TInner, + } -use std::fmt::{Formatter, Error, Debug}; - -use crate::progress::Timestamp; -use crate::progress::timestamp::Refines; + impl Product { + /// Creates a new product from outer and inner coordinates. + pub fn new(outer: TOuter, inner: TInner) -> Self { + Product { + outer, + inner, + } + } + } -impl Refines for Product { - fn to_inner(other: TOuter) -> Self { - Product::new(other, TInner::minimum()) + // Debug implementation to avoid seeing fully qualified path names. + impl Debug for Product { + fn fmt(&self, f: &mut Formatter) -> Result<(), Error> { + f.write_str(&format!("({:?}, {:?})", self.outer, self.inner)) + } } - fn to_outer(self: Product) -> TOuter { - self.outer + + use super::PartialOrder; + impl PartialOrder for Product { + #[inline] + fn less_equal(&self, other: &Self) -> bool { + self.outer.less_equal(&other.outer) && self.inner.less_equal(&other.inner) + } } - fn summarize(path: ::Summary) -> ::Summary { - path.outer + + impl Timestamp for Product { + type Summary = Product; + fn minimum() -> Self { Self { outer: TOuter::minimum(), inner: TInner::minimum() }} } -} -/// A nested pair of timestamps, one outer and one inner. -/// -/// We use `Product` rather than `(TOuter, TInner)` so that we can derive our own `PartialOrd`, -/// because Rust just uses the lexicographic total order. -#[derive(Abomonation, Copy, Clone, Hash, Eq, PartialEq, Default, Ord, PartialOrd, Serialize, Deserialize)] -pub struct Product { - /// Outer timestamp. - pub outer: TOuter, - /// Inner timestamp. - pub inner: TInner, -} + impl PathSummary> for Product { + #[inline] + fn results_in(&self, product: &Product) -> Option> { + self.outer.results_in(&product.outer) + .and_then(|outer| + self.inner.results_in(&product.inner) + .map(|inner| Product::new(outer, inner)) + ) + } + #[inline] + fn followed_by(&self, other: &Self) -> Option { + self.outer.followed_by(&other.outer) + .and_then(|outer| + self.inner.followed_by(&other.inner) + .map(|inner| Product::new(outer, inner)) + ) + } + } -impl Product { - /// Creates a new product from outer and inner coordinates. - pub fn new(outer: TOuter, inner: TInner) -> Product { - Product { - outer, - inner, + impl Refines for Product { + fn to_inner(other: TOuter) -> Self { + Product::new(other, TInner::minimum()) + } + fn to_outer(self: Product) -> TOuter { + self.outer + } + fn summarize(path: ::Summary) -> ::Summary { + path.outer } } -} -/// Debug implementation to avoid seeing fully qualified path names. -impl Debug for Product { - fn fmt(&self, f: &mut Formatter) -> Result<(), Error> { - f.write_str(&format!("({:?}, {:?})", self.outer, self.inner)) + impl Empty for Product { } + impl TotalOrder for Product where T1: Empty, T2: TotalOrder { } + + impl Columnation for Product { + type InnerRegion = ProductRegion; } -} -impl PartialOrder for Product { - #[inline] - fn less_equal(&self, other: &Self) -> bool { - self.outer.less_equal(&other.outer) && self.inner.less_equal(&other.inner) + #[derive(Default)] + pub struct ProductRegion { + outer_region: T1, + inner_region: T2, } -} -impl Timestamp for Product { - type Summary = Product; - fn minimum() -> Self { Product { outer: TOuter::minimum(), inner: TInner::minimum() }} + impl Region for ProductRegion { + type Item = Product; + + #[inline] + unsafe fn copy(&mut self, item: &Self::Item) -> Self::Item { + Product { outer: self.outer_region.copy(&item.outer), inner: self.inner_region.copy(&item.inner) } + } + + fn clear(&mut self) { + self.outer_region.clear(); + self.inner_region.clear(); + } + + fn reserve_items<'a, I>(&mut self, items1: I) where Self: 'a, I: Iterator + Clone { + let items2 = items1.clone(); + self.outer_region.reserve_items(items1.map(|x| &x.outer)); + self.inner_region.reserve_items(items2.map(|x| &x.inner)) + } + + fn reserve_regions<'a, I>(&mut self, regions1: I) where Self: 'a, I: Iterator + Clone { + let regions2 = regions1.clone(); + self.outer_region.reserve_regions(regions1.map(|r| &r.outer_region)); + self.inner_region.reserve_regions(regions2.map(|r| &r.inner_region)); + } + + fn heap_size(&self, mut callback: impl FnMut(usize, usize)) { + self.outer_region.heap_size(&mut callback); + self.inner_region.heap_size(callback); + } + } } -use crate::progress::timestamp::PathSummary; -impl PathSummary> for Product { - #[inline] - fn results_in(&self, product: &Product) -> Option> { - self.outer.results_in(&product.outer) - .and_then(|outer| - self.inner.results_in(&product.inner) - .map(|inner| Product::new(outer, inner)) - ) +/// Rust tuple ordered by the lexicographic order. +mod tuple { + + use super::PartialOrder; + impl PartialOrder for (TOuter, TInner) { + #[inline] + fn less_equal(&self, other: &Self) -> bool { + // We avoid Rust's `PartialOrd` implementation, for reasons of correctness. + self.0.less_than(&other.0) || (self.0.eq(&other.0) && self.1.less_equal(&other.1)) + } } - #[inline] - fn followed_by(&self, other: &Product) -> Option> { - self.outer.followed_by(&other.outer) - .and_then(|outer| - self.inner.followed_by(&other.inner) - .map(|inner| Product::new(outer, inner)) - ) + + use super::TotalOrder; + impl TotalOrder for (T1, T2) where T1: TotalOrder, T2: TotalOrder { } + + use crate::progress::Timestamp; + impl Timestamp for (TOuter, TInner) { + type Summary = (TOuter::Summary, TInner::Summary); + fn minimum() -> Self { (TOuter::minimum(), TInner::minimum()) } } -} -/// A type that does not affect total orderedness. -/// -/// This trait is not useful, but must be made public and documented or else Rust -/// complains about its existence in the constraints on the implementation of -/// public traits for public types. -pub trait Empty : PartialOrder { } + use crate::progress::timestamp::PathSummary; + impl PathSummary<(TOuter, TInner)> for (TOuter::Summary, TInner::Summary) { + #[inline] + fn results_in(&self, (outer, inner): &(TOuter, TInner)) -> Option<(TOuter, TInner)> { + self.0.results_in(outer) + .and_then(|outer| + self.1.results_in(inner) + .map(|inner| (outer, inner)) + ) + } + #[inline] + fn followed_by(&self, (outer, inner): &(TOuter::Summary, TInner::Summary)) -> Option<(TOuter::Summary, TInner::Summary)> { + self.0.followed_by(outer) + .and_then(|outer| + self.1.followed_by(inner) + .map(|inner| (outer, inner)) + ) + } + } -impl Empty for () { } -impl Empty for Product { } + use crate::progress::timestamp::Refines; + impl Refines for (TOuter, TInner) { + fn to_inner(other: TOuter) -> Self { + (other, TInner::minimum()) + } + fn to_outer(self: (TOuter, TInner)) -> TOuter { + self.0 + } + fn summarize(path: ::Summary) -> ::Summary { + path.0 + } + } -impl TotalOrder for Product where T1: Empty, T2: TotalOrder { } + use super::Empty; + impl Empty for (T1, T2) { } +} diff --git a/timely/src/progress/frontier.rs b/timely/src/progress/frontier.rs index ecba92310..ba44de481 100644 --- a/timely/src/progress/frontier.rs +++ b/timely/src/progress/frontier.rs @@ -1,7 +1,7 @@ //! Tracks minimal sets of mutually incomparable elements of a partial order. use crate::progress::ChangeBatch; -use crate::order::PartialOrder; +use crate::order::{PartialOrder, TotalOrder}; /// A set of mutually incomparable elements. /// @@ -13,7 +13,7 @@ use crate::order::PartialOrder; /// Two antichains are equal if the contain the same set of elements, even if in different orders. /// This can make equality testing quadratic, though linear in the common case that the sequences /// are identical. -#[derive(Debug, Default, Abomonation, Serialize, Deserialize)] +#[derive(Debug, Abomonation, Serialize, Deserialize)] pub struct Antichain { elements: Vec } @@ -43,6 +43,32 @@ impl Antichain { } } + /// Updates the `Antichain` if the element is not greater than or equal to some present element. + /// + /// Returns true if element is added to the set + /// + /// Accepts a reference to an element, which is cloned when inserting. + /// + /// # Examples + /// + ///``` + /// use timely::progress::frontier::Antichain; + /// + /// let mut frontier = Antichain::new(); + /// assert!(frontier.insert_ref(&2)); + /// assert!(!frontier.insert(3)); + ///``` + pub fn insert_ref(&mut self, element: &T) -> bool where T: Clone { + if !self.elements.iter().any(|x| x.less_equal(element)) { + self.elements.retain(|x| !element.less_equal(x)); + self.elements.push(element.clone()); + true + } + else { + false + } + } + /// Reserves capacity for at least additional more elements to be inserted in the given `Antichain` pub fn reserve(&mut self, additional: usize) { self.elements.reserve(additional); @@ -116,6 +142,17 @@ impl Antichain { } } +impl std::iter::FromIterator for Antichain { + fn from_iter(iterator: I) -> Self + where + I: IntoIterator + { + let mut result = Self::new(); + result.extend(iterator); + result + } +} + impl Antichain { /// Creates a new empty `Antichain`. @@ -227,6 +264,37 @@ impl Clone for Antichain { } } +impl Default for Antichain { + fn default() -> Self { + Self::new() + } +} + +impl TotalOrder for Antichain { } + +impl Antichain { + /// Convert to the at most one element the antichain contains. + pub fn into_option(mut self) -> Option { + debug_assert!(self.len() <= 1); + self.elements.pop() + } + /// Return a reference to the at most one element the antichain contains. + pub fn as_option(&self) -> Option<&T> { + debug_assert!(self.len() <= 1); + self.elements.last() + } +} + +impl std::hash::Hash for Antichain { + fn hash(&self, state: &mut H) { + let mut temp = self.elements.iter().collect::>(); + temp.sort(); + for element in temp { + element.hash(state); + } + } +} + impl From> for Antichain { fn from(vec: Vec) -> Self { // TODO: We could reuse `vec` with some care. @@ -272,14 +340,9 @@ impl ::std::iter::IntoIterator for Antichain { /// The `MutableAntichain` implementation is done with the intent that updates to it are done in batches, /// and it is acceptable to rebuild the frontier from scratch when a batch of updates change it. This means /// that it can be expensive to maintain a large number of counts and change few elements near the frontier. -/// -/// There is an `update_dirty` method for single updates that leave the `MutableAntichain` in a dirty state, -/// but I strongly recommend against using them unless you must (on part of timely progress tracking seems -/// to be greatly simplified by access to this) #[derive(Clone, Debug, Abomonation, Serialize, Deserialize)] pub struct MutableAntichain { - dirty: usize, - updates: Vec<(T, i64)>, + updates: ChangeBatch, frontier: Vec, changes: ChangeBatch, } @@ -298,8 +361,7 @@ impl MutableAntichain { #[inline] pub fn new() -> MutableAntichain { MutableAntichain { - dirty: 0, - updates: Vec::new(), + updates: ChangeBatch::new(), frontier: Vec::new(), changes: ChangeBatch::new(), } @@ -318,21 +380,11 @@ impl MutableAntichain { ///``` #[inline] pub fn clear(&mut self) { - self.dirty = 0; self.updates.clear(); self.frontier.clear(); self.changes.clear(); } - /// This method deletes the contents. Unlike `clear` it records doing so. - pub fn empty(&mut self) { - for (_, diff) in self.updates.iter_mut() { - *diff = 0; - } - - self.dirty = self.updates.len(); - } - /// Reveals the minimal elements with positive count. /// /// # Examples @@ -345,7 +397,6 @@ impl MutableAntichain { ///``` #[inline] pub fn frontier(&self) -> AntichainRef<'_, T> { - debug_assert_eq!(self.dirty, 0); AntichainRef::new(&self.frontier) } @@ -360,13 +411,12 @@ impl MutableAntichain { /// assert!(frontier.frontier() == AntichainRef::new(&[0u64])); ///``` #[inline] - pub fn new_bottom(bottom: T) -> MutableAntichain + pub fn new_bottom(bottom: T) -> MutableAntichain where - T: Clone, + T: Ord+Clone, { MutableAntichain { - dirty: 0, - updates: vec![(bottom.clone(), 1)], + updates: ChangeBatch::new_from(bottom.clone(), 1), frontier: vec![bottom], changes: ChangeBatch::new(), } @@ -384,7 +434,6 @@ impl MutableAntichain { ///``` #[inline] pub fn is_empty(&self) -> bool { - debug_assert_eq!(self.dirty, 0); self.frontier.is_empty() } @@ -405,7 +454,6 @@ impl MutableAntichain { where T: PartialOrder, { - debug_assert_eq!(self.dirty, 0); self.frontier().less_than(time) } @@ -426,22 +474,9 @@ impl MutableAntichain { where T: PartialOrder, { - debug_assert_eq!(self.dirty, 0); self.frontier().less_equal(time) } - /// Allows a single-element push, but dirties the antichain and prevents inspection until cleaned. - /// - /// At the moment inspection is prevented via panic, so best be careful (this should probably be fixed). - /// It is *very* important if you want to use this method that very soon afterwards you call something - /// akin to `update_iter`, perhaps with a `None` argument if you have no more data, as this method will - /// tidy up the internal representation. - #[inline] - pub fn update_dirty(&mut self, time: T, delta: i64) { - self.updates.push((time, delta)); - self.dirty += 1; - } - /// Applies updates to the antichain and enumerates any changes. /// /// # Examples @@ -466,32 +501,20 @@ impl MutableAntichain { { let updates = updates.into_iter(); - // Attempt to pre-allocate for the new updates - let (min, max) = updates.size_hint(); - self.updates.reserve(max.unwrap_or(min)); - - for (time, delta) in updates { - self.updates.push((time, delta)); - self.dirty += 1; - } - // track whether a rebuild is needed. let mut rebuild_required = false; + for (time, delta) in updates { - // determine if recently pushed data requires rebuilding the frontier. - // note: this may be required even with an empty iterator, due to dirty data in self.updates. - while self.dirty > 0 && !rebuild_required { - - let time = &self.updates[self.updates.len() - self.dirty].0; - let delta = self.updates[self.updates.len() - self.dirty].1; - - let beyond_frontier = self.frontier.iter().any(|f| f.less_than(time)); - let before_frontier = !self.frontier.iter().any(|f| f.less_equal(time)); - rebuild_required = rebuild_required || !(beyond_frontier || (delta < 0 && before_frontier)); + // If we do not yet require a rebuild, test whether we might require one + // and set the flag in that case. + if !rebuild_required { + let beyond_frontier = self.frontier.iter().any(|f| f.less_than(&time)); + let before_frontier = !self.frontier.iter().any(|f| f.less_equal(&time)); + rebuild_required = !(beyond_frontier || (delta < 0 && before_frontier)); + } - self.dirty -= 1; + self.updates.update(time, delta); } - self.dirty = 0; if rebuild_required { self.rebuild() @@ -499,7 +522,7 @@ impl MutableAntichain { self.changes.drain() } - /// Sorts and consolidates `self.updates` and applies `action` to any frontier changes. + /// Rebuilds `self.frontier` from `self.updates`. /// /// This method is meant to be used for bulk updates to the frontier, and does more work than one might do /// for single updates, but is meant to be an efficient way to process multiple updates together. This is @@ -508,19 +531,6 @@ impl MutableAntichain { where T: Clone + PartialOrder + Ord, { - - // sort and consolidate updates; retain non-zero accumulations. - if !self.updates.is_empty() { - self.updates.sort_by(|x,y| x.0.cmp(&y.0)); - for i in 0 .. self.updates.len() - 1 { - if self.updates[i].0 == self.updates[i+1].0 { - self.updates[i+1].1 += self.updates[i].1; - self.updates[i].1 = 0; - } - } - self.updates.retain(|x| x.1 != 0); - } - for time in self.frontier.drain(..) { self.changes.update(time, -1); } @@ -544,11 +554,23 @@ impl MutableAntichain { T: Ord, { self.updates + .unstable_internal_updates() .iter() .filter(|td| td.0.eq(query_time)) .map(|td| td.1) .sum() } + + /// Reports the updates that form the frontier. Returns an iterator of timestamps and their frequency. + /// + /// Rebuilds the internal representation before revealing times and frequencies. + pub fn updates(&mut self) -> impl Iterator + where + T: Clone + PartialOrder + Ord, + { + self.rebuild(); + self.updates.iter() + } } impl Default for MutableAntichain { @@ -583,6 +605,35 @@ impl> MutableAntichainF } } +impl From> for MutableAntichain { + fn from(antichain: Antichain) -> Self { + let mut result = MutableAntichain::new(); + result.update_iter(antichain.into_iter().map(|time| (time, 1))); + result + } +} +impl<'a, T: PartialOrder+Ord+Clone> From> for MutableAntichain { + fn from(antichain: AntichainRef<'a, T>) -> Self { + let mut result = MutableAntichain::new(); + result.update_iter(antichain.into_iter().map(|time| (time.clone(), 1))); + result + } +} + +impl std::iter::FromIterator<(T, i64)> for MutableAntichain +where + T: Clone + PartialOrder + Ord, +{ + fn from_iter(iterator: I) -> Self + where + I: IntoIterator, + { + let mut result = Self::new(); + result.update_iter(iterator); + result + } +} + /// A wrapper for elements of an antichain. #[derive(Debug)] pub struct AntichainRef<'a, T: 'a> { @@ -684,6 +735,16 @@ impl<'a, T: PartialOrder> PartialOrder for AntichainRef<'a, T> { } } +impl<'a, T: TotalOrder> TotalOrder for AntichainRef<'a, T> { } + +impl<'a, T: TotalOrder> AntichainRef<'a, T> { + /// Return a reference to the at most one element the antichain contains. + pub fn as_option(&self) -> Option<&T> { + debug_assert!(self.len() <= 1); + self.frontier.last() + } +} + impl<'a, T> ::std::ops::Deref for AntichainRef<'a, T> { type Target = [T]; fn deref(&self) -> &Self::Target { @@ -698,3 +759,56 @@ impl<'a, T: 'a> ::std::iter::IntoIterator for &'a AntichainRef<'a, T> { self.iter() } } + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use super::*; + + #[derive(PartialEq, Eq, PartialOrd, Ord, Hash)] + struct Elem(char, usize); + + impl PartialOrder for Elem { + fn less_equal(&self, other: &Self) -> bool { + self.0 <= other.0 && self.1 <= other.1 + } + } + + #[test] + fn antichain_hash() { + let mut hashed = HashSet::new(); + hashed.insert(Antichain::from(vec![Elem('a', 2), Elem('b', 1)])); + + assert!(hashed.contains(&Antichain::from(vec![Elem('a', 2), Elem('b', 1)]))); + assert!(hashed.contains(&Antichain::from(vec![Elem('b', 1), Elem('a', 2)]))); + + assert!(!hashed.contains(&Antichain::from(vec![Elem('a', 2)]))); + assert!(!hashed.contains(&Antichain::from(vec![Elem('a', 1)]))); + assert!(!hashed.contains(&Antichain::from(vec![Elem('b', 2)]))); + assert!(!hashed.contains(&Antichain::from(vec![Elem('a', 1), Elem('b', 2)]))); + assert!(!hashed.contains(&Antichain::from(vec![Elem('c', 3)]))); + assert!(!hashed.contains(&Antichain::from(vec![]))); + } + + #[test] + fn mutable_compaction() { + let mut mutable = MutableAntichain::new(); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((7, 1))); + mutable.update_iter(Some((8, 1))); + mutable.update_iter(Some((8, 1))); + mutable.update_iter(Some((8, 1))); + mutable.update_iter(Some((8, 1))); + mutable.update_iter(Some((8, 1))); + for _ in 0 .. 1000 { + mutable.update_iter(Some((9, 1))); + mutable.update_iter(Some((9, -1))); + } + assert!(mutable.updates.unstable_internal_updates().len() <= 32); + } +} diff --git a/timely/src/progress/reachability.rs b/timely/src/progress/reachability.rs index 8e51c839f..5a1f1f6f6 100644 --- a/timely/src/progress/reachability.rs +++ b/timely/src/progress/reachability.rs @@ -897,4 +897,48 @@ pub mod logging { impl From for TrackerEvent { fn from(v: TargetUpdate) -> TrackerEvent { TrackerEvent::TargetUpdate(v) } } -} \ No newline at end of file +} + +// The Drop implementation for `Tracker` makes sure that reachability logging is correct for +// prematurely dropped dataflows. At the moment, this is only possible through `drop_dataflow`, +// because in all other cases the tracker stays alive while it has outstanding work, leaving no +// remaining work for this Drop implementation. +impl Drop for Tracker { + fn drop(&mut self) { + let logger = if let Some(logger) = &mut self.logger { + logger + } else { + // No cleanup necessary when there is no logger. + return; + }; + + // Retract pending data that `propagate_all` would normally log. + for (index, per_operator) in self.per_operator.iter_mut().enumerate() { + let target_changes = per_operator.targets + .iter_mut() + .enumerate() + .flat_map(|(port, target)| { + target.pointstamps + .updates() + .map(move |(time, diff)| (index, port, time.clone(), -diff)) + }) + .collect::>(); + if !target_changes.is_empty() { + logger.log_target_updates(Box::new(target_changes)); + } + + let source_changes = per_operator.sources + .iter_mut() + .enumerate() + .flat_map(|(port, source)| { + source.pointstamps + .updates() + .map(move |(time, diff)| (index, port, time.clone(), -diff)) + }) + .collect::>(); + if !source_changes.is_empty() { + logger.log_source_updates(Box::new(source_changes)); + } + } + } +} diff --git a/timely/src/progress/subgraph.rs b/timely/src/progress/subgraph.rs index 309d9077c..f41fee190 100644 --- a/timely/src/progress/subgraph.rs +++ b/timely/src/progress/subgraph.rs @@ -200,6 +200,7 @@ where incomplete_count, activations, temp_active: BinaryHeap::new(), + maybe_shutdown: Vec::new(), children: self.children, input_messages: self.input_messages, output_capabilities: self.output_capabilities, @@ -242,6 +243,7 @@ where // shared activations (including children). activations: Rc>, temp_active: BinaryHeap>, + maybe_shutdown: Vec, // shared state written to by the datapath, counting records entering this subgraph instance. input_messages: Vec>>>, @@ -461,6 +463,7 @@ where // Drain propagated information into shared progress structure. for ((location, time), diff) in self.pointstamp_tracker.pushed().drain() { + self.maybe_shutdown.push(location.node); // Targets are actionable, sources are not. if let crate::progress::Port::Target(port) = location.port { if self.children[location.node].notify { @@ -477,6 +480,18 @@ where } } + // Consider scheduling each recipient of progress information to shut down. + self.maybe_shutdown.sort(); + self.maybe_shutdown.dedup(); + for child_index in self.maybe_shutdown.drain(..) { + let child_state = self.pointstamp_tracker.node_state(child_index); + let frontiers_empty = child_state.targets.iter().all(|x| x.implications.is_empty()); + let no_capabilities = child_state.sources.iter().all(|x| x.pointstamps.is_empty()); + if frontiers_empty && no_capabilities { + self.temp_active.push(Reverse(child_index)); + } + } + // Extract child zero frontier changes and report as internal capability changes. for (output, internal) in self.shared_progress.borrow_mut().internals.iter_mut().enumerate() { self.pointstamp_tracker diff --git a/timely/src/scheduling/activate.rs b/timely/src/scheduling/activate.rs index b4f7ff412..e86f44355 100644 --- a/timely/src/scheduling/activate.rs +++ b/timely/src/scheduling/activate.rs @@ -1,14 +1,12 @@ //! Parking and unparking timely fibers. use std::rc::Rc; -use std::sync::Arc; use std::cell::RefCell; use std::thread::Thread; use std::collections::BinaryHeap; use std::time::{Duration, Instant}; use std::cmp::Reverse; use crossbeam_channel::{Sender, Receiver}; -use futures_util::task::ArcWake; /// Methods required to act as a timely scheduler. /// @@ -187,7 +185,7 @@ impl Activations { } /// A thread-safe handle to an `Activations`. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct SyncActivations { tx: Sender>, thread: Thread, @@ -218,7 +216,7 @@ impl SyncActivations { } /// A capability to activate a specific path. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct Activator { path: Vec, queue: Rc>, @@ -253,7 +251,7 @@ impl Activator { } /// A thread-safe version of `Activator`. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct SyncActivator { path: Vec, queue: SyncActivations, @@ -274,15 +272,9 @@ impl SyncActivator { } } -impl ArcWake for SyncActivator { - fn wake_by_ref(arc_self: &Arc) { - arc_self.activate().unwrap(); - } -} - /// The error returned when activation fails across thread boundaries because /// the receiving end has hung up. -#[derive(Debug)] +#[derive(Clone, Copy, Debug)] pub struct SyncActivationError; impl std::fmt::Display for SyncActivationError { @@ -291,14 +283,10 @@ impl std::fmt::Display for SyncActivationError { } } -impl std::error::Error for SyncActivationError { - fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { - None - } -} +impl std::error::Error for SyncActivationError {} /// A wrapper that unparks on drop. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct ActivateOnDrop { wrapped: T, address: Rc>, diff --git a/timely/src/synchronization/sequence.rs b/timely/src/synchronization/sequence.rs index 7525d3389..320a3dcb8 100644 --- a/timely/src/synchronization/sequence.rs +++ b/timely/src/synchronization/sequence.rs @@ -157,7 +157,7 @@ impl Sequencer { } let mut activator_borrow = activator_source.borrow_mut(); - let mut activator = activator_borrow.as_mut().unwrap(); + let activator = activator_borrow.as_mut().unwrap(); if let Some(t) = activator.catchup_until { if capability.time().less_than(&t) { @@ -190,7 +190,7 @@ impl Sequencer { if let Some(last) = recvd.last() { let mut activator_borrow = activator_sink.borrow_mut(); - let mut activator = activator_borrow.as_mut().unwrap(); + let activator = activator_borrow.as_mut().unwrap(); activator.catchup_until = Some((last.0).0); activator.activate(); diff --git a/timely/src/worker.rs b/timely/src/worker.rs index d3d01fa2b..3db8d9733 100644 --- a/timely/src/worker.rs +++ b/timely/src/worker.rs @@ -92,7 +92,7 @@ pub struct Config { } impl Config { - /// Installs options into a [`getopts::Options`] struct that correspond + /// Installs options into a [getopts_dep::Options] struct that correspond /// to the parameters in the configuration. /// /// It is the caller's responsibility to ensure that the installed options @@ -109,8 +109,8 @@ impl Config { /// Instantiates a configuration based upon the parsed options in `matches`. /// /// The `matches` object must have been constructed from a - /// [`getopts::Options`] which contained at least the options installed by - /// [`Self::install_options`]. + /// [getopts_dep::Options] which contained at least the options installed by + /// [Self::install_options]. /// /// This method is only available if the `getopts` feature is enabled, which /// it is by default. @@ -154,7 +154,7 @@ impl Config { /// Gets the value for configured parameter `key`. /// /// Returns `None` if `key` has not previously been set with - /// [`WorkerConfig::set`], or if the specified `T` does not match the `T` + /// [Config::set], or if the specified `T` does not match the `T` /// from the call to `set`. /// /// # Examples @@ -338,8 +338,9 @@ impl Worker { let events = allocator.events().clone(); let mut borrow = events.borrow_mut(); let paths = self.paths.borrow(); - for (channel, _event) in borrow.drain(..) { - // TODO: Pay more attent to `_event`. + borrow.sort_unstable(); + borrow.dedup(); + for channel in borrow.drain(..) { // Consider tracking whether a channel // in non-empty, and only activating // on the basis of non-empty channels. @@ -366,7 +367,7 @@ impl Worker { (x, y) => x.or(y), }; - if !self.dataflows.borrow().is_empty() && delay != Some(Duration::new(0,0)) { + if delay != Some(Duration::new(0,0)) { // Log parking and flush log. if let Some(l) = self.logging().as_mut() { @@ -698,6 +699,11 @@ impl Worker { self.dataflows.borrow().keys().cloned().collect() } + /// True if there is at least one dataflow under management. + pub fn has_dataflows(&self) -> bool { + !self.dataflows.borrow().is_empty() + } + // Acquire a new distinct dataflow identifier. fn allocate_dataflow_index(&mut self) -> usize { *self.dataflow_counter.borrow_mut() += 1; diff --git a/timely/tests/gh_523.rs b/timely/tests/gh_523.rs new file mode 100644 index 000000000..9003f7afa --- /dev/null +++ b/timely/tests/gh_523.rs @@ -0,0 +1,41 @@ +use timely::dataflow::channels::pact::Pipeline; +use timely::dataflow::operators::{Exchange, Input, Operator, Probe}; +use timely::dataflow::InputHandle; +use timely::Config; + +#[test] +fn gh_523() { + timely::execute(Config::thread(), |worker| { + let mut input = InputHandle::new(); + let mut buf = Vec::new(); + let probe = worker.dataflow::(|scope| { + scope + .input_from(&mut input) + .unary(Pipeline, "Test", move |_, _| { + move |input, output| { + input.for_each(|cap, data| { + data.swap(&mut buf); + let mut session = output.session(&cap); + session.give_container(&mut Vec::new()); + session.give_container(&mut buf); + }); + } + }) + .exchange(|x| *x) + .probe() + }); + + for round in 0..2 { + input.send(round); + input.advance_to(round + 1); + } + input.close(); + + while !probe.done() { + worker.step(); + } + + println!("worker {} complete", worker.index()); + }) + .unwrap(); +}