├── .gitignore ├── .travis.yml ├── CHANGELOG.md ├── CONTRIBUTING.md ├── COPYRIGHT ├── Cargo.toml ├── LICENSE ├── README.md ├── bytes ├── .gitignore ├── Cargo.toml └── src │ └── lib.rs ├── communication ├── .gitignore ├── Cargo.toml ├── examples │ └── comm_hello.rs └── src │ ├── allocator │ ├── canary.rs │ ├── counters.rs │ ├── generic.rs │ ├── mod.rs │ ├── process.rs │ ├── thread.rs │ └── zero_copy │ │ ├── allocator.rs │ │ ├── allocator_process.rs │ │ ├── bytes_exchange.rs │ │ ├── bytes_slab.rs │ │ ├── initialize.rs │ │ ├── mod.rs │ │ ├── push_pull.rs │ │ └── tcp.rs │ ├── buzzer.rs │ ├── initialize.rs │ ├── lib.rs │ ├── logging.rs │ ├── message.rs │ └── networking.rs ├── kafkaesque ├── .gitignore ├── Cargo.toml └── src │ ├── bin │ ├── capture_recv.rs │ ├── capture_send.rs │ └── kafka_source.rs │ ├── kafka_source.rs │ └── lib.rs ├── logging ├── .gitignore ├── Cargo.toml └── src │ └── lib.rs ├── mdbook ├── .gitignore └── src │ ├── SUMMARY.md │ ├── chapter_0 │ ├── chapter_0.md │ ├── chapter_0_0.md │ ├── chapter_0_1.md │ ├── chapter_0_2.md │ └── chapter_0_3.md │ ├── chapter_1 │ ├── chapter_1.md │ ├── chapter_1_1.md │ ├── chapter_1_2.md │ └── chapter_1_3.md │ ├── chapter_2 │ ├── chapter_2.md │ ├── chapter_2_1.md │ ├── chapter_2_2.md │ ├── chapter_2_3.md │ ├── chapter_2_4.md │ └── chapter_2_5.md │ ├── chapter_3 │ ├── chapter_3.md │ ├── chapter_3_1.md │ ├── chapter_3_2.md │ ├── chapter_3_3.md │ └── chapter_3_4.md │ ├── chapter_4 │ ├── chapter_4.md │ ├── chapter_4_1.md │ ├── chapter_4_2.md │ ├── chapter_4_3.md │ ├── chapter_4_4.md │ └── chapter_4_5.md │ ├── chapter_5 │ ├── chapter_5.md │ ├── chapter_5_1.md │ └── chapter_5_2.md │ └── introduction.md ├── sort ├── .gitignore ├── Cargo.toml ├── LICENSE ├── README.md ├── benches │ └── benches.rs └── src │ ├── batched_vec.rs │ ├── bin │ └── profile.rs │ ├── lib.rs │ ├── lsb.rs │ ├── lsb_swc.rs │ ├── msb.rs │ ├── msb_swc.rs │ ├── stash.rs │ └── swc_buffer.rs └── timely ├── Cargo.toml ├── examples ├── barrier.rs ├── bfs.rs ├── capture_recv.rs ├── capture_send.rs ├── distinct.rs ├── event_driven.rs ├── exchange.rs ├── flow_controlled.rs ├── hashjoin.rs ├── hello.rs ├── logging-recv.rs ├── logging-send.rs ├── loopdemo.rs ├── openloop.rs ├── pagerank.rs ├── pingpong.rs ├── rc.rs ├── sequence.rs ├── simple.rs ├── threadless.rs ├── unionfind.rs ├── unordered_input.rs └── wordcount.rs ├── src ├── dataflow │ ├── channels │ │ ├── mod.rs │ │ ├── pact.rs │ │ ├── pullers │ │ │ ├── counter.rs │ │ │ └── mod.rs │ │ └── pushers │ │ │ ├── buffer.rs │ │ │ ├── counter.rs │ │ │ ├── exchange.rs │ │ │ ├── mod.rs │ │ │ └── tee.rs │ ├── mod.rs │ ├── operators │ │ ├── aggregation │ │ │ ├── aggregate.rs │ │ │ ├── mod.rs │ │ │ └── state_machine.rs │ │ ├── branch.rs │ │ ├── broadcast.rs │ │ ├── capability.rs │ │ ├── capture │ │ │ ├── capture.rs │ │ │ ├── event.rs │ │ │ ├── extract.rs │ │ │ ├── mod.rs │ │ │ └── replay.rs │ │ ├── concat.rs │ │ ├── count.rs │ │ ├── delay.rs │ │ ├── enterleave.rs │ │ ├── exchange.rs │ │ ├── feedback.rs │ │ ├── filter.rs │ │ ├── flow_controlled.rs │ │ ├── generic │ │ │ ├── binary.rs │ │ │ ├── builder_raw.rs │ │ │ ├── builder_rc.rs │ │ │ ├── builder_ref.rs │ │ │ ├── handles.rs │ │ │ ├── mod.rs │ │ │ ├── notificator.rs │ │ │ ├── operator.rs │ │ │ ├── operator_info.rs │ │ │ └── unary.rs │ │ ├── input.rs │ │ ├── inspect.rs │ │ ├── map.rs │ │ ├── mod.rs │ │ ├── partition.rs │ │ ├── probe.rs │ │ ├── queue.rs │ │ ├── reclock.rs │ │ ├── to_stream.rs │ │ └── unordered_input.rs │ ├── scopes │ │ ├── child.rs │ │ └── mod.rs │ └── stream.rs ├── execute.rs ├── lib.rs ├── logging.rs ├── order.rs ├── progress │ ├── broadcast.rs │ ├── change_batch.rs │ ├── frontier.rs │ ├── mod.rs │ ├── operate.rs │ ├── reachability.rs │ ├── subgraph.rs │ └── timestamp.rs ├── scheduling │ ├── activate.rs │ └── mod.rs ├── synchronization │ ├── barrier.rs │ ├── mod.rs │ └── sequence.rs └── worker.rs └── tests ├── barrier.rs └── skeptic.rs /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /.vscode 3 | Cargo.lock 4 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | dist: xenial 2 | language: rust 3 | rust: 4 | - stable 5 | install: 6 | - pip install --user ghp-import 7 | script: 8 | - cargo build 9 | # rustdoc doesn't build dependencies, so it needs to run after `cargo build`, 10 | # but its dependency search gets confused if there are multiple copies of any 11 | # dependency in target/debug/deps, so it needs to run before `cargo test` et al. 12 | # clutter target/debug/deps with multiple copies of things. 13 | - for file in $(find mdbook -name '*.md'); do rustdoc --test $file -L ./target/debug/deps; done 14 | - cargo test 15 | - cargo bench 16 | - cargo doc 17 | after_success: | 18 | [ $TRAVIS_BRANCH = master ] && 19 | [ $TRAVIS_PULL_REQUEST = false ] && 20 | cargo install mdbook && 21 | (cd mdbook; mdbook build) && 22 | ghp-import -n mdbook/book && 23 | git push -fq https://${GH_TOKEN}@github.com/${TRAVIS_REPO_SLUG}.git gh-pages 24 | env: 25 | global: 26 | - secure: CzDrbiXWbvgjqZn1z5aFcttW90tWT4LpEc9bSJ/Wb0i4QvABAaOefw3Griu9MT9vNvbyM1ysk8fzXMHXF1ivhUZFzV4MRr+nrKg4LTg/upDAVAEOjWdrFUJOp17a+PVS3K83zrVxeZMkmQe4ACB3a9hsHilKp635xgizSJWQ6Wo= 27 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | Thank you for your interest in contributing! 2 | 3 | Here is some legal stuff that will make you regret clicking on this link. 4 | 5 | By submitting a pull request for this project, you are agreeing to license your contribution under the terms of the project's LICENSE file at the time of your submission (in case it changes or something). You are also certifying that you are in a position to make this agreement, in that you didn't nick your code from someone else, or some project with conflicting licensing requirements. 6 | 7 | If you would like to put explicit copyright notices somewhere, please leave them in the repository's COPYRIGHT file rather than in each file. 8 | -------------------------------------------------------------------------------- /COPYRIGHT: -------------------------------------------------------------------------------- 1 | Contributions by Andrea Lattuada are Copyright (c) 2016 Andrea Lattuada, ETH Zürich. 2 | Contributions by Moritz Hoffmann are Copyright (c) 2017 Moritz Hoffmann, ETH Zürich. 3 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [workspace] 2 | members = [ 3 | "bytes", 4 | "communication", 5 | "kafkaesque", 6 | "logging", 7 | "sort", 8 | "timely", 9 | ] 10 | 11 | [profile.release] 12 | opt-level = 3 13 | debug = true 14 | rpath = false 15 | lto = true 16 | debug-assertions = false 17 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2014 Frank McSherry 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | 23 | -------------------------------------------------------------------------------- /bytes/.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /.vscode 3 | Cargo.lock 4 | -------------------------------------------------------------------------------- /bytes/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "timely_bytes" 3 | version = "0.10.0" 4 | authors = ["Frank McSherry "] 5 | edition = "2018" 6 | 7 | description = "Disjoint mutable byte slices from a common allocation" 8 | 9 | documentation = "https://docs.rs/timely/" 10 | homepage = "https://github.com/TimelyDataflow/timely-dataflow" 11 | repository = "https://github.com/TimelyDataflow/timely-dataflow.git" 12 | keywords = ["timely", "dataflow", "bytes"] 13 | license = "MIT" -------------------------------------------------------------------------------- /communication/.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | Cargo.lock 3 | -------------------------------------------------------------------------------- /communication/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "timely_communication" 3 | version = "0.10.0" 4 | authors = ["Frank McSherry "] 5 | description = "Communication layer for timely dataflow" 6 | edition = "2018" 7 | 8 | # These URLs point to more information about the repository 9 | 10 | documentation = "https://docs.rs/timely/" 11 | homepage = "https://github.com/TimelyDataflow/timely-dataflow" 12 | repository = "https://github.com/TimelyDataflow/timely-dataflow.git" 13 | keywords = ["timely", "dataflow"] 14 | license = "MIT" 15 | 16 | [features] 17 | default = ["getopts"] 18 | 19 | [dependencies] 20 | getopts = { version = "0.2.14", optional = true} 21 | bincode = { version = "1.0", optional = true } 22 | serde_derive = "1.0" 23 | serde = "1.0" 24 | abomonation = "0.7" 25 | abomonation_derive = "0.4" 26 | timely_bytes = { path = "../bytes", version = "0.10" } 27 | timely_logging = { path = "../logging", version = "0.10" } 28 | -------------------------------------------------------------------------------- /communication/examples/comm_hello.rs: -------------------------------------------------------------------------------- 1 | extern crate timely_communication; 2 | 3 | use std::ops::Deref; 4 | use timely_communication::{Message, Allocate}; 5 | 6 | fn main() { 7 | 8 | // extract the configuration from user-supplied arguments, initialize the computation. 9 | let config = timely_communication::Configuration::from_args(std::env::args()).unwrap(); 10 | let guards = timely_communication::initialize(config, |mut allocator| { 11 | 12 | println!("worker {} of {} started", allocator.index(), allocator.peers()); 13 | 14 | // allocates pair of senders list and one receiver. 15 | let (mut senders, mut receiver) = allocator.allocate(0); 16 | 17 | // send typed data along each channel 18 | for i in 0 .. allocator.peers() { 19 | senders[i].send(Message::from_typed(format!("hello, {}", i))); 20 | senders[i].done(); 21 | } 22 | 23 | // no support for termination notification, 24 | // we have to count down ourselves. 25 | let mut received = 0; 26 | while received < allocator.peers() { 27 | 28 | allocator.receive(); 29 | 30 | if let Some(message) = receiver.recv() { 31 | println!("worker {}: received: <{}>", allocator.index(), message.deref()); 32 | received += 1; 33 | } 34 | 35 | allocator.release(); 36 | } 37 | 38 | allocator.index() 39 | }); 40 | 41 | // computation runs until guards are joined or dropped. 42 | if let Ok(guards) = guards { 43 | for guard in guards.join() { 44 | println!("result: {:?}", guard); 45 | } 46 | } 47 | else { println!("error in computation"); } 48 | } 49 | -------------------------------------------------------------------------------- /communication/src/allocator/canary.rs: -------------------------------------------------------------------------------- 1 | //! A helper struct to report when something has been dropped. 2 | 3 | use std::rc::Rc; 4 | use std::cell::RefCell; 5 | 6 | /// An opaque type that reports when it is dropped. 7 | pub struct Canary { 8 | index: usize, 9 | queue: Rc>>, 10 | } 11 | 12 | impl Canary { 13 | /// Allocates a new drop canary. 14 | pub fn new(index: usize, queue: Rc>>) -> Self { 15 | Canary { index, queue } 16 | } 17 | } 18 | 19 | impl Drop for Canary { 20 | fn drop(&mut self) { 21 | self.queue.borrow_mut().push(self.index); 22 | } 23 | } -------------------------------------------------------------------------------- /communication/src/allocator/counters.rs: -------------------------------------------------------------------------------- 1 | //! Push and Pull wrappers to maintain counts of messages in channels. 2 | 3 | use std::rc::Rc; 4 | use std::cell::RefCell; 5 | use std::collections::VecDeque; 6 | 7 | use crate::{Push, Pull}; 8 | use crate::allocator::Event; 9 | 10 | /// The push half of an intra-thread channel. 11 | pub struct Pusher> { 12 | index: usize, 13 | // count: usize, 14 | events: Rc>>, 15 | pusher: P, 16 | phantom: ::std::marker::PhantomData, 17 | } 18 | 19 | impl> Pusher { 20 | /// Wraps a pusher with a message counter. 21 | pub fn new(pusher: P, index: usize, events: Rc>>) -> Self { 22 | Pusher { 23 | index, 24 | // count: 0, 25 | events, 26 | pusher, 27 | phantom: ::std::marker::PhantomData, 28 | } 29 | } 30 | } 31 | 32 | impl> Push for Pusher { 33 | #[inline] 34 | fn push(&mut self, element: &mut Option) { 35 | // if element.is_none() { 36 | // if self.count != 0 { 37 | // self.events 38 | // .borrow_mut() 39 | // .push_back((self.index, Event::Pushed(self.count))); 40 | // self.count = 0; 41 | // } 42 | // } 43 | // else { 44 | // self.count += 1; 45 | // } 46 | // TODO: Version above is less chatty, but can be a bit late in 47 | // moving information along. Better, but needs cooperation. 48 | self.events 49 | .borrow_mut() 50 | .push_back((self.index, Event::Pushed(1))); 51 | 52 | self.pusher.push(element) 53 | } 54 | } 55 | 56 | use std::sync::mpsc::Sender; 57 | 58 | /// The push half of an intra-thread channel. 59 | pub struct ArcPusher> { 60 | index: usize, 61 | // count: usize, 62 | events: Sender<(usize, Event)>, 63 | pusher: P, 64 | phantom: ::std::marker::PhantomData, 65 | } 66 | 67 | impl> ArcPusher { 68 | /// Wraps a pusher with a message counter. 69 | pub fn new(pusher: P, index: usize, events: Sender<(usize, Event)>) -> Self { 70 | ArcPusher { 71 | index, 72 | // count: 0, 73 | events, 74 | pusher, 75 | phantom: ::std::marker::PhantomData, 76 | } 77 | } 78 | } 79 | 80 | impl> Push for ArcPusher { 81 | #[inline] 82 | fn push(&mut self, element: &mut Option) { 83 | // if element.is_none() { 84 | // if self.count != 0 { 85 | // self.events 86 | // .send((self.index, Event::Pushed(self.count))) 87 | // .expect("Failed to send message count"); 88 | // self.count = 0; 89 | // } 90 | // } 91 | // else { 92 | // self.count += 1; 93 | // } 94 | let _ = 95 | self.events 96 | .send((self.index, Event::Pushed(1))); 97 | // TODO : Perhaps this shouldn't be a fatal error (e.g. in shutdown). 98 | // .expect("Failed to send message count"); 99 | 100 | self.pusher.push(element) 101 | } 102 | } 103 | 104 | /// The pull half of an intra-thread channel. 105 | pub struct Puller> { 106 | index: usize, 107 | count: usize, 108 | events: Rc>>, 109 | puller: P, 110 | phantom: ::std::marker::PhantomData, 111 | } 112 | 113 | impl> Puller { 114 | /// Wraps a puller with a message counter. 115 | pub fn new(puller: P, index: usize, events: Rc>>) -> Self { 116 | Puller { 117 | index, 118 | count: 0, 119 | events, 120 | puller, 121 | phantom: ::std::marker::PhantomData, 122 | } 123 | } 124 | } 125 | impl> Pull for Puller { 126 | #[inline] 127 | fn pull(&mut self) -> &mut Option { 128 | let result = self.puller.pull(); 129 | if result.is_none() { 130 | if self.count != 0 { 131 | self.events 132 | .borrow_mut() 133 | .push_back((self.index, Event::Pulled(self.count))); 134 | self.count = 0; 135 | } 136 | } 137 | else { 138 | self.count += 1; 139 | } 140 | 141 | result 142 | } 143 | } 144 | -------------------------------------------------------------------------------- /communication/src/allocator/mod.rs: -------------------------------------------------------------------------------- 1 | //! Types and traits for the allocation of channels. 2 | 3 | use std::rc::Rc; 4 | use std::cell::RefCell; 5 | use std::time::Duration; 6 | use std::collections::VecDeque; 7 | 8 | pub use self::thread::Thread; 9 | pub use self::process::Process; 10 | pub use self::generic::{Generic, GenericBuilder}; 11 | 12 | pub mod thread; 13 | pub mod process; 14 | pub mod generic; 15 | 16 | pub mod canary; 17 | pub mod counters; 18 | 19 | pub mod zero_copy; 20 | 21 | use crate::{Data, Push, Pull, Message}; 22 | 23 | /// A proto-allocator, which implements `Send` and can be completed with `build`. 24 | /// 25 | /// This trait exists because some allocators contain elements that do not implement 26 | /// the `Send` trait, for example `Rc` wrappers for shared state. As such, what we 27 | /// actually need to create to initialize a computation are builders, which we can 28 | /// then move into new threads each of which then construct their actual allocator. 29 | pub trait AllocateBuilder : Send { 30 | /// The type of allocator to be built. 31 | type Allocator: Allocate; 32 | /// Builds allocator, consumes self. 33 | fn build(self) -> Self::Allocator; 34 | } 35 | 36 | /// A type capable of allocating channels. 37 | /// 38 | /// There is some feature creep, in that this contains several convenience methods about the nature 39 | /// of the allocated channels, and maintenance methods to ensure that they move records around. 40 | pub trait Allocate { 41 | /// The index of the worker out of `(0..self.peers())`. 42 | fn index(&self) -> usize; 43 | /// The number of workers in the communication group. 44 | fn peers(&self) -> usize; 45 | /// Constructs several send endpoints and one receive endpoint. 46 | fn allocate(&mut self, identifier: usize) -> (Vec>>>, Box>>); 47 | /// A shared queue of communication events with channel identifier. 48 | /// 49 | /// It is expected that users of the channel allocator will regularly 50 | /// drain these events in order to drive their computation. If they 51 | /// fail to do so the event queue may become quite large, and turn 52 | /// into a performance problem. 53 | fn events(&self) -> &Rc>>; 54 | 55 | /// Awaits communication events. 56 | /// 57 | /// This method may park the current thread, for at most `duration`, 58 | /// until new events arrive. 59 | /// The method is not guaranteed to wait for any amount of time, but 60 | /// good implementations should use this as a hint to park the thread. 61 | fn await_events(&self, _duration: Option) { } 62 | 63 | /// Ensure that received messages are surfaced in each channel. 64 | /// 65 | /// This method should be called to ensure that received messages are 66 | /// surfaced in each channel, but failing to call the method does not 67 | /// ensure that they are not surfaced. 68 | /// 69 | /// Generally, this method is the indication that the allocator should 70 | /// present messages contained in otherwise scarce resources (for example 71 | /// network buffers), under the premise that someone is about to consume 72 | /// the messages and release the resources. 73 | fn receive(&mut self) { } 74 | 75 | /// Signal the completion of a batch of reads from channels. 76 | /// 77 | /// Conventionally, this method signals to the communication fabric 78 | /// that the worker is taking a break from reading from channels, and 79 | /// the fabric should consider re-acquiring scarce resources. This can 80 | /// lead to the fabric performing defensive copies out of un-consumed 81 | /// buffers, and can be a performance problem if invoked casually. 82 | fn release(&mut self) { } 83 | 84 | /// Constructs a pipeline channel from the worker to itself. 85 | /// 86 | /// By default, this method uses the thread-local channel constructor 87 | /// based on a shared `VecDeque` which updates the event queue. 88 | fn pipeline(&mut self, identifier: usize) -> 89 | (thread::ThreadPusher>, 90 | thread::ThreadPuller>) 91 | { 92 | thread::Thread::new_from(identifier, self.events().clone()) 93 | } 94 | } 95 | 96 | /// A communication channel event. 97 | pub enum Event { 98 | /// A number of messages pushed into the channel. 99 | Pushed(usize), 100 | /// A number of messages pulled from the channel. 101 | Pulled(usize), 102 | } 103 | -------------------------------------------------------------------------------- /communication/src/allocator/thread.rs: -------------------------------------------------------------------------------- 1 | //! Intra-thread communication. 2 | 3 | use std::rc::Rc; 4 | use std::cell::RefCell; 5 | use std::time::Duration; 6 | use std::collections::VecDeque; 7 | 8 | use crate::allocator::{Allocate, AllocateBuilder, Event}; 9 | use crate::allocator::counters::Pusher as CountPusher; 10 | use crate::allocator::counters::Puller as CountPuller; 11 | use crate::{Push, Pull, Message}; 12 | 13 | /// Builder for single-threaded allocator. 14 | pub struct ThreadBuilder; 15 | 16 | impl AllocateBuilder for ThreadBuilder { 17 | type Allocator = Thread; 18 | fn build(self) -> Self::Allocator { Thread::new() } 19 | } 20 | 21 | 22 | /// An allocator for intra-thread communication. 23 | pub struct Thread { 24 | /// Shared counts of messages in channels. 25 | events: Rc>>, 26 | } 27 | 28 | impl Allocate for Thread { 29 | fn index(&self) -> usize { 0 } 30 | fn peers(&self) -> usize { 1 } 31 | fn allocate(&mut self, identifier: usize) -> (Vec>>>, Box>>) { 32 | let (pusher, puller) = Thread::new_from(identifier, self.events.clone()); 33 | (vec![Box::new(pusher)], Box::new(puller)) 34 | } 35 | fn events(&self) -> &Rc>> { 36 | &self.events 37 | } 38 | fn await_events(&self, duration: Option) { 39 | if self.events.borrow().is_empty() { 40 | if let Some(duration) = duration { 41 | std::thread::park_timeout(duration); 42 | } 43 | else { 44 | std::thread::park(); 45 | } 46 | } 47 | } 48 | } 49 | 50 | /// Thread-local counting channel push endpoint. 51 | pub type ThreadPusher = CountPusher>; 52 | /// Thread-local counting channel pull endpoint. 53 | pub type ThreadPuller = CountPuller>; 54 | 55 | impl Thread { 56 | /// Allocates a new thread-local channel allocator. 57 | pub fn new() -> Self { 58 | Thread { 59 | events: Rc::new(RefCell::new(VecDeque::new())), 60 | } 61 | } 62 | 63 | /// Creates a new thread-local channel from an identifier and shared counts. 64 | pub fn new_from(identifier: usize, events: Rc>>) 65 | -> (ThreadPusher>, ThreadPuller>) 66 | { 67 | let shared = Rc::new(RefCell::new((VecDeque::>::new(), VecDeque::>::new()))); 68 | let pusher = Pusher { target: shared.clone() }; 69 | let pusher = CountPusher::new(pusher, identifier, events.clone()); 70 | let puller = Puller { source: shared, current: None }; 71 | let puller = CountPuller::new(puller, identifier, events.clone()); 72 | (pusher, puller) 73 | } 74 | } 75 | 76 | 77 | /// The push half of an intra-thread channel. 78 | pub struct Pusher { 79 | target: Rc, VecDeque)>>, 80 | } 81 | 82 | impl Push for Pusher { 83 | #[inline] 84 | fn push(&mut self, element: &mut Option) { 85 | let mut borrow = self.target.borrow_mut(); 86 | if let Some(element) = element.take() { 87 | borrow.0.push_back(element); 88 | } 89 | *element = borrow.1.pop_front(); 90 | } 91 | } 92 | 93 | /// The pull half of an intra-thread channel. 94 | pub struct Puller { 95 | current: Option, 96 | source: Rc, VecDeque)>>, 97 | } 98 | 99 | impl Pull for Puller { 100 | #[inline] 101 | fn pull(&mut self) -> &mut Option { 102 | let mut borrow = self.source.borrow_mut(); 103 | // if let Some(element) = self.current.take() { 104 | // // TODO : Arbitrary constant. 105 | // if borrow.1.len() < 16 { 106 | // borrow.1.push_back(element); 107 | // } 108 | // } 109 | self.current = borrow.0.pop_front(); 110 | &mut self.current 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /communication/src/allocator/zero_copy/bytes_slab.rs: -------------------------------------------------------------------------------- 1 | //! A large binary allocation for writing and sharing. 2 | 3 | use bytes::arc::Bytes; 4 | 5 | /// A large binary allocation for writing and sharing. 6 | /// 7 | /// A bytes slab wraps a `Bytes` and maintains a valid (written) length, and supports writing after 8 | /// this valid length, and extracting `Bytes` up to this valid length. Extracted bytes are enqueued 9 | /// and checked for uniqueness in order to recycle them (once all shared references are dropped). 10 | pub struct BytesSlab { 11 | buffer: Bytes, // current working buffer. 12 | in_progress: Vec>, // buffers shared with workers. 13 | stash: Vec, // reclaimed and resuable buffers. 14 | shift: usize, // current buffer allocation size. 15 | valid: usize, // buffer[..valid] are valid bytes. 16 | } 17 | 18 | impl BytesSlab { 19 | /// Allocates a new `BytesSlab` with an initial size determined by a shift. 20 | pub fn new(shift: usize) -> Self { 21 | BytesSlab { 22 | buffer: Bytes::from(vec![0u8; 1 << shift].into_boxed_slice()), 23 | in_progress: Vec::new(), 24 | stash: Vec::new(), 25 | shift, 26 | valid: 0, 27 | } 28 | } 29 | /// The empty region of the slab. 30 | pub fn empty(&mut self) -> &mut [u8] { 31 | &mut self.buffer[self.valid..] 32 | } 33 | /// The valid region of the slab. 34 | pub fn valid(&mut self) -> &mut [u8] { 35 | &mut self.buffer[..self.valid] 36 | } 37 | /// Marks the next `bytes` bytes as valid. 38 | pub fn make_valid(&mut self, bytes: usize) { 39 | self.valid += bytes; 40 | } 41 | /// Extracts the first `bytes` valid bytes. 42 | pub fn extract(&mut self, bytes: usize) -> Bytes { 43 | debug_assert!(bytes <= self.valid); 44 | self.valid -= bytes; 45 | self.buffer.extract_to(bytes) 46 | } 47 | 48 | /// Ensures that `self.empty().len()` is at least `capacity`. 49 | /// 50 | /// This method may retire the current buffer if it does not have enough space, in which case 51 | /// it will copy any remaining contents into a new buffer. If this would not create enough free 52 | /// space, the shift is increased until it is sufficient. 53 | pub fn ensure_capacity(&mut self, capacity: usize) { 54 | 55 | if self.empty().len() < capacity { 56 | 57 | let mut increased_shift = false; 58 | 59 | // Increase allocation if copy would be insufficient. 60 | while self.valid + capacity > (1 << self.shift) { 61 | self.shift += 1; 62 | self.stash.clear(); // clear wrongly sized buffers. 63 | self.in_progress.clear(); // clear wrongly sized buffers. 64 | increased_shift = true; 65 | } 66 | 67 | // Attempt to reclaim shared slices. 68 | if self.stash.is_empty() { 69 | for shared in self.in_progress.iter_mut() { 70 | if let Some(mut bytes) = shared.take() { 71 | if bytes.try_regenerate::>() { 72 | // NOTE: Test should be redundant, but better safe... 73 | if bytes.len() == (1 << self.shift) { 74 | self.stash.push(bytes); 75 | } 76 | } 77 | else { 78 | *shared = Some(bytes); 79 | } 80 | } 81 | } 82 | self.in_progress.retain(|x| x.is_some()); 83 | } 84 | 85 | let new_buffer = self.stash.pop().unwrap_or_else(|| Bytes::from(vec![0; 1 << self.shift].into_boxed_slice())); 86 | let old_buffer = ::std::mem::replace(&mut self.buffer, new_buffer); 87 | 88 | self.buffer[.. self.valid].copy_from_slice(&old_buffer[.. self.valid]); 89 | if !increased_shift { 90 | self.in_progress.push(Some(old_buffer)); 91 | } 92 | } 93 | } 94 | } -------------------------------------------------------------------------------- /communication/src/allocator/zero_copy/initialize.rs: -------------------------------------------------------------------------------- 1 | //! Network initialization. 2 | 3 | use std::sync::Arc; 4 | // use crate::allocator::Process; 5 | use crate::allocator::process::ProcessBuilder; 6 | use crate::networking::create_sockets; 7 | use super::tcp::{send_loop, recv_loop}; 8 | use super::allocator::{TcpBuilder, new_vector}; 9 | 10 | /// Join handles for send and receive threads. 11 | /// 12 | /// On drop, the guard joins with each of the threads to ensure that they complete 13 | /// cleanly and send all necessary data. 14 | pub struct CommsGuard { 15 | send_guards: Vec<::std::thread::JoinHandle<()>>, 16 | recv_guards: Vec<::std::thread::JoinHandle<()>>, 17 | } 18 | 19 | impl Drop for CommsGuard { 20 | fn drop(&mut self) { 21 | for handle in self.send_guards.drain(..) { 22 | handle.join().expect("Send thread panic"); 23 | } 24 | // println!("SEND THREADS JOINED"); 25 | for handle in self.recv_guards.drain(..) { 26 | handle.join().expect("Recv thread panic"); 27 | } 28 | // println!("RECV THREADS JOINED"); 29 | } 30 | } 31 | 32 | use crate::logging::{CommunicationSetup, CommunicationEvent}; 33 | use logging_core::Logger; 34 | 35 | /// Initializes network connections 36 | pub fn initialize_networking( 37 | addresses: Vec, 38 | my_index: usize, 39 | threads: usize, 40 | noisy: bool, 41 | log_sender: BoxOption>+Send+Sync>) 42 | -> ::std::io::Result<(Vec>, CommsGuard)> 43 | { 44 | let log_sender = Arc::new(log_sender); 45 | let processes = addresses.len(); 46 | 47 | // one per process (including local, which would be None) 48 | let mut results: Vec> = 49 | create_sockets(addresses, my_index, noisy)?; 50 | 51 | let process_allocators = crate::allocator::process::Process::new_vector(threads); 52 | let (builders, promises, futures) = new_vector(process_allocators, my_index, processes); 53 | 54 | let mut promises_iter = promises.into_iter(); 55 | let mut futures_iter = futures.into_iter(); 56 | 57 | let mut send_guards = Vec::new(); 58 | let mut recv_guards = Vec::new(); 59 | 60 | // for each process, if a stream exists (i.e. not local) ... 61 | for index in 0..results.len() { 62 | 63 | if let Some(stream) = results[index].take() { 64 | // remote process 65 | 66 | let remote_recv = promises_iter.next().unwrap(); 67 | 68 | { 69 | let log_sender = log_sender.clone(); 70 | let stream = stream.try_clone()?; 71 | let join_guard = 72 | ::std::thread::Builder::new() 73 | .name(format!("send thread {}", index)) 74 | .spawn(move || { 75 | 76 | let logger = log_sender(CommunicationSetup { 77 | process: my_index, 78 | sender: true, 79 | remote: Some(index), 80 | }); 81 | 82 | send_loop(stream, remote_recv, my_index, index, logger); 83 | })?; 84 | 85 | send_guards.push(join_guard); 86 | } 87 | 88 | let remote_send = futures_iter.next().unwrap(); 89 | 90 | { 91 | // let remote_sends = remote_sends.clone(); 92 | let log_sender = log_sender.clone(); 93 | let stream = stream.try_clone()?; 94 | let join_guard = 95 | ::std::thread::Builder::new() 96 | .name(format!("recv thread {}", index)) 97 | .spawn(move || { 98 | let logger = log_sender(CommunicationSetup { 99 | process: my_index, 100 | sender: false, 101 | remote: Some(index), 102 | }); 103 | recv_loop(stream, remote_send, threads * my_index, my_index, index, logger); 104 | })?; 105 | 106 | recv_guards.push(join_guard); 107 | } 108 | 109 | } 110 | } 111 | 112 | Ok((builders, CommsGuard { send_guards, recv_guards })) 113 | } -------------------------------------------------------------------------------- /communication/src/allocator/zero_copy/mod.rs: -------------------------------------------------------------------------------- 1 | //! Allocators based on serialized data which avoid copies. 2 | //! 3 | //! These allocators are based on `Abomonation` serialization, and its ability to deserialized 4 | //! typed Rust data in-place. They surface references to data, often ultimately referencing the 5 | //! raw binary data they initial received. 6 | 7 | pub mod bytes_slab; 8 | pub mod bytes_exchange; 9 | pub mod tcp; 10 | pub mod allocator; 11 | pub mod allocator_process; 12 | pub mod initialize; 13 | pub mod push_pull; -------------------------------------------------------------------------------- /communication/src/allocator/zero_copy/push_pull.rs: -------------------------------------------------------------------------------- 1 | //! Push and Pull implementations wrapping serialized data. 2 | 3 | use std::rc::Rc; 4 | use std::cell::RefCell; 5 | use std::collections::VecDeque; 6 | 7 | use bytes::arc::Bytes; 8 | 9 | use crate::allocator::canary::Canary; 10 | use crate::networking::MessageHeader; 11 | 12 | use crate::{Data, Push, Pull}; 13 | use crate::allocator::Message; 14 | 15 | use super::bytes_exchange::{BytesPush, SendEndpoint}; 16 | 17 | /// An adapter into which one may push elements of type `T`. 18 | /// 19 | /// This pusher has a fixed MessageHeader, and access to a SharedByteBuffer which it uses to 20 | /// acquire buffers for serialization. 21 | pub struct Pusher { 22 | header: MessageHeader, 23 | sender: Rc>>, 24 | phantom: ::std::marker::PhantomData, 25 | } 26 | 27 | impl Pusher { 28 | /// Creates a new `Pusher` from a header and shared byte buffer. 29 | pub fn new(header: MessageHeader, sender: Rc>>) -> Pusher { 30 | Pusher { 31 | header: header, 32 | sender: sender, 33 | phantom: ::std::marker::PhantomData, 34 | } 35 | } 36 | } 37 | 38 | impl Push> for Pusher { 39 | #[inline] 40 | fn push(&mut self, element: &mut Option>) { 41 | if let Some(ref mut element) = *element { 42 | 43 | // determine byte lengths and build header. 44 | let mut header = self.header; 45 | self.header.seqno += 1; 46 | header.length = element.length_in_bytes(); 47 | assert!(header.length > 0); 48 | 49 | // acquire byte buffer and write header, element. 50 | let mut borrow = self.sender.borrow_mut(); 51 | { 52 | let mut bytes = borrow.reserve(header.required_bytes()); 53 | assert!(bytes.len() >= header.required_bytes()); 54 | let writer = &mut bytes; 55 | header.write_to(writer).expect("failed to write header!"); 56 | element.into_bytes(writer); 57 | } 58 | borrow.make_valid(header.required_bytes()); 59 | } 60 | } 61 | } 62 | 63 | /// An adapter from which one can pull elements of type `T`. 64 | /// 65 | /// This type is very simple, and just consumes owned `Vec` allocations. It is 66 | /// not the most efficient thing possible, which would probably instead be something 67 | /// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared 68 | /// allocation. 69 | pub struct Puller { 70 | _canary: Canary, 71 | current: Option>, 72 | receiver: Rc>>, // source of serialized buffers 73 | } 74 | 75 | impl Puller { 76 | /// Creates a new `Puller` instance from a shared queue. 77 | pub fn new(receiver: Rc>>, _canary: Canary) -> Puller { 78 | Puller { 79 | _canary, 80 | current: None, 81 | receiver, 82 | } 83 | } 84 | } 85 | 86 | impl Pull> for Puller { 87 | #[inline] 88 | fn pull(&mut self) -> &mut Option> { 89 | self.current = 90 | self.receiver 91 | .borrow_mut() 92 | .pop_front() 93 | .map(|bytes| unsafe { Message::from_bytes(bytes) }); 94 | 95 | &mut self.current 96 | } 97 | } 98 | 99 | /// An adapter from which one can pull elements of type `T`. 100 | /// 101 | /// This type is very simple, and just consumes owned `Vec` allocations. It is 102 | /// not the most efficient thing possible, which would probably instead be something 103 | /// like the `bytes` crate (../bytes/) which provides an exclusive view of a shared 104 | /// allocation. 105 | pub struct PullerInner { 106 | inner: Box>>, // inner pullable (e.g. intra-process typed queue) 107 | _canary: Canary, 108 | current: Option>, 109 | receiver: Rc>>, // source of serialized buffers 110 | } 111 | 112 | impl PullerInner { 113 | /// Creates a new `PullerInner` instance from a shared queue. 114 | pub fn new(inner: Box>>, receiver: Rc>>, _canary: Canary) -> Self { 115 | PullerInner { 116 | inner, 117 | _canary, 118 | current: None, 119 | receiver, 120 | } 121 | } 122 | } 123 | 124 | impl Pull> for PullerInner { 125 | #[inline] 126 | fn pull(&mut self) -> &mut Option> { 127 | 128 | let inner = self.inner.pull(); 129 | if inner.is_some() { 130 | inner 131 | } 132 | else { 133 | self.current = 134 | self.receiver 135 | .borrow_mut() 136 | .pop_front() 137 | .map(|bytes| unsafe { Message::from_bytes(bytes) }); 138 | 139 | &mut self.current 140 | } 141 | } 142 | } -------------------------------------------------------------------------------- /communication/src/buzzer.rs: -------------------------------------------------------------------------------- 1 | //! A type that can unpark specific threads. 2 | 3 | use std::thread::Thread; 4 | 5 | /// Can unpark a specific thread. 6 | #[derive(Clone)] 7 | pub struct Buzzer { 8 | thread: Thread, 9 | } 10 | 11 | impl Buzzer { 12 | /// Creates a new buzzer for the current thread. 13 | pub fn new() -> Self { 14 | Self { 15 | thread: std::thread::current() 16 | } 17 | } 18 | /// Unparks the target thread. 19 | pub fn buzz(&self) { 20 | self.thread.unpark() 21 | } 22 | } -------------------------------------------------------------------------------- /communication/src/logging.rs: -------------------------------------------------------------------------------- 1 | //! Configuration and events for communication logging. 2 | 3 | /// Configuration information about a communication thread. 4 | #[derive(Abomonation, Debug, PartialEq, Eq, Hash, Clone, Copy)] 5 | pub struct CommunicationSetup { 6 | /// True when this is a send thread (or the receive thread). 7 | pub sender: bool, 8 | /// The process id of the thread. 9 | pub process: usize, 10 | /// The remote process id. 11 | pub remote: Option, 12 | } 13 | 14 | /// Various communication events. 15 | #[derive(Abomonation, Debug, PartialEq, Eq, Hash, Clone, Copy)] 16 | pub enum CommunicationEvent { 17 | /// An observed message. 18 | Message(MessageEvent), 19 | /// A state transition. 20 | State(StateEvent), 21 | } 22 | 23 | /// An observed message. 24 | #[derive(Abomonation, Debug, PartialEq, Eq, Hash, Clone, Copy)] 25 | pub struct MessageEvent { 26 | /// true for send event, false for receive event 27 | pub is_send: bool, 28 | /// associated message header. 29 | pub header: crate::networking::MessageHeader, 30 | } 31 | 32 | /// Starting or stopping communication threads. 33 | #[derive(Abomonation, Debug, PartialEq, Eq, Hash, Clone, Copy)] 34 | pub struct StateEvent { 35 | /// Is the thread a send (vs a recv) thread. 36 | pub send: bool, 37 | /// The host process id. 38 | pub process: usize, 39 | /// The remote process id. 40 | pub remote: usize, 41 | /// Is the thread starting or stopping. 42 | pub start: bool, 43 | } 44 | 45 | impl From for CommunicationEvent { 46 | fn from(v: MessageEvent) -> CommunicationEvent { CommunicationEvent::Message(v) } 47 | } 48 | impl From for CommunicationEvent { 49 | fn from(v: StateEvent) -> CommunicationEvent { CommunicationEvent::State(v) } 50 | } 51 | -------------------------------------------------------------------------------- /kafkaesque/.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /.vscode 3 | Cargo.lock 4 | -------------------------------------------------------------------------------- /kafkaesque/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "kafkaesque" 3 | version = "0.1.0" 4 | authors = ["Frank McSherry "] 5 | edition = "2018" 6 | 7 | [dependencies] 8 | clap="*" 9 | abomonation="0.7" 10 | timely = { path = "../timely" } 11 | 12 | [dependencies.rdkafka] 13 | version = "0.20.0" 14 | -------------------------------------------------------------------------------- /kafkaesque/src/bin/capture_recv.rs: -------------------------------------------------------------------------------- 1 | use timely::dataflow::operators::Inspect; 2 | use timely::dataflow::operators::capture::Replay; 3 | use timely::dataflow::operators::Accumulate; 4 | 5 | use rdkafka::config::ClientConfig; 6 | 7 | use kafkaesque::EventConsumer; 8 | 9 | fn main() { 10 | timely::execute_from_args(std::env::args(), |worker| { 11 | 12 | let topic = std::env::args().nth(1).unwrap(); 13 | let source_peers = std::env::args().nth(2).unwrap().parse::().unwrap(); 14 | let brokers = "localhost:9092"; 15 | 16 | // Create Kafka stuff. 17 | let mut consumer_config = ClientConfig::new(); 18 | consumer_config 19 | .set("produce.offset.report", "true") 20 | .set("auto.offset.reset", "smallest") 21 | .set("group.id", "example") 22 | .set("enable.auto.commit", "false") 23 | .set("enable.partition.eof", "false") 24 | .set("auto.offset.reset", "earliest") 25 | .set("session.timeout.ms", "6000") 26 | .set("bootstrap.servers", &brokers); 27 | 28 | // create replayers from disjoint partition of source worker identifiers. 29 | let replayers = 30 | (0 .. source_peers) 31 | .filter(|i| i % worker.peers() == worker.index()) 32 | .map(|i| { 33 | let topic = format!("{}-{:?}", topic, i); 34 | EventConsumer::<_,u64>::new(consumer_config.clone(), topic) 35 | }) 36 | .collect::>(); 37 | 38 | worker.dataflow::(|scope| { 39 | replayers 40 | .replay_into(scope) 41 | .count() 42 | .inspect(|x| println!("replayed: {:?}", x)) 43 | ; 44 | }) 45 | }).unwrap(); // asserts error-free execution 46 | } 47 | -------------------------------------------------------------------------------- /kafkaesque/src/bin/capture_send.rs: -------------------------------------------------------------------------------- 1 | use timely::dataflow::operators::ToStream; 2 | use timely::dataflow::operators::capture::Capture; 3 | 4 | use rdkafka::config::ClientConfig; 5 | 6 | use kafkaesque::EventProducer; 7 | 8 | fn main() { 9 | timely::execute_from_args(std::env::args(), |worker| { 10 | 11 | // target topic name. 12 | let topic = std::env::args().nth(1).unwrap(); 13 | let count = std::env::args().nth(2).unwrap().parse::().unwrap(); 14 | let brokers = "localhost:9092"; 15 | 16 | // Create Kafka stuff. 17 | let mut producer_config = ClientConfig::new(); 18 | producer_config 19 | .set("produce.offset.report", "true") 20 | .set("bootstrap.servers", brokers); 21 | 22 | let topic = format!("{}-{:?}", topic, worker.index()); 23 | let producer = EventProducer::new(producer_config, topic); 24 | 25 | worker.dataflow::(|scope| 26 | (0 .. count) 27 | .to_stream(scope) 28 | .capture_into(producer) 29 | ); 30 | }).unwrap(); 31 | } 32 | -------------------------------------------------------------------------------- /kafkaesque/src/bin/kafka_source.rs: -------------------------------------------------------------------------------- 1 | use timely::dataflow::operators::Inspect; 2 | 3 | use rdkafka::config::ClientConfig; 4 | use rdkafka::consumer::{Consumer, BaseConsumer, DefaultConsumerContext}; 5 | 6 | fn main() { 7 | 8 | let mut args = ::std::env::args(); 9 | args.next(); 10 | 11 | // Extract Kafka topic. 12 | let topic = args.next().expect("Must specify a Kafka topic"); 13 | let brokers = "localhost:9092"; 14 | 15 | // Create Kafka consumer configuration. 16 | // Feel free to change parameters here. 17 | let mut consumer_config = ClientConfig::new(); 18 | consumer_config 19 | .set("produce.offset.report", "true") 20 | .set("auto.offset.reset", "smallest") 21 | .set("group.id", "example") 22 | .set("enable.auto.commit", "false") 23 | .set("enable.partition.eof", "false") 24 | .set("auto.offset.reset", "earliest") 25 | .set("session.timeout.ms", "6000") 26 | .set("bootstrap.servers", &brokers); 27 | 28 | timely::execute_from_args(args, move |worker| { 29 | 30 | // A dataflow for producing spans. 31 | worker.dataflow::(|scope| { 32 | 33 | // Create a Kafka consumer. 34 | let consumer : BaseConsumer = consumer_config.create().expect("Couldn't create consumer"); 35 | consumer.subscribe(&[&topic]).expect("Failed to subscribe to topic"); 36 | 37 | let strings = 38 | kafkaesque::source(scope, "KafkaStringSource", consumer, |bytes, capability, output| { 39 | 40 | // If the bytes are utf8, convert to string and send. 41 | if let Ok(text) = std::str::from_utf8(bytes) { 42 | output 43 | .session(capability) 44 | .give(text.to_string()); 45 | } 46 | 47 | // We need some rule to advance timestamps ... 48 | let time = *capability.time(); 49 | capability.downgrade(&(time + 1)); 50 | 51 | // Indicate that we are not yet done. 52 | false 53 | }); 54 | 55 | strings.inspect(|x| println!("Observed: {:?}", x)); 56 | 57 | }); 58 | 59 | }).expect("Timely computation failed somehow"); 60 | 61 | println!("Hello, world!"); 62 | } 63 | 64 | -------------------------------------------------------------------------------- /kafkaesque/src/lib.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | use std::sync::atomic::{AtomicIsize, Ordering}; 3 | 4 | use abomonation::Abomonation; 5 | use timely::dataflow::operators::capture::event::{Event, EventPusher, EventIterator}; 6 | 7 | use rdkafka::Message; 8 | use rdkafka::client::ClientContext; 9 | use rdkafka::config::ClientConfig; 10 | use rdkafka::producer::{BaseProducer, BaseRecord, ProducerContext, DeliveryResult}; 11 | use rdkafka::consumer::{Consumer, BaseConsumer, DefaultConsumerContext}; 12 | 13 | use rdkafka::config::FromClientConfigAndContext; 14 | 15 | pub mod kafka_source; 16 | pub use kafka_source::kafka_source as source; 17 | 18 | struct OutstandingCounterContext { 19 | outstanding: Arc, 20 | } 21 | 22 | impl ClientContext for OutstandingCounterContext { } 23 | 24 | impl ProducerContext for OutstandingCounterContext { 25 | type DeliveryOpaque = (); 26 | fn delivery(&self, _report: &DeliveryResult, _: Self::DeliveryOpaque) { 27 | self.outstanding.fetch_sub(1, Ordering::SeqCst); 28 | } 29 | } 30 | 31 | impl OutstandingCounterContext { 32 | pub fn new(counter: &Arc) -> Self { 33 | OutstandingCounterContext { 34 | outstanding: counter.clone() 35 | } 36 | } 37 | } 38 | 39 | /// A wrapper for `W: Write` implementing `EventPusher`. 40 | pub struct EventProducer { 41 | topic: String, 42 | buffer: Vec, 43 | producer: BaseProducer, 44 | counter: Arc, 45 | phant: ::std::marker::PhantomData<(T,D)>, 46 | } 47 | 48 | impl EventProducer { 49 | /// Allocates a new `EventWriter` wrapping a supplied writer. 50 | pub fn new(config: ClientConfig, topic: String) -> Self { 51 | let counter = Arc::new(AtomicIsize::new(0)); 52 | let context = OutstandingCounterContext::new(&counter); 53 | let producer = BaseProducer::::from_config_and_context(&config, context).expect("Couldn't create producer"); 54 | println!("allocating producer for topic {:?}", topic); 55 | EventProducer { 56 | topic: topic, 57 | buffer: vec![], 58 | producer: producer, 59 | counter: counter, 60 | phant: ::std::marker::PhantomData, 61 | } 62 | } 63 | } 64 | 65 | impl EventPusher for EventProducer { 66 | fn push(&mut self, event: Event) { 67 | unsafe { ::abomonation::encode(&event, &mut self.buffer).expect("Encode failure"); } 68 | // println!("sending {:?} bytes", self.buffer.len()); 69 | self.producer.send::<(),[u8]>(BaseRecord::to(self.topic.as_str()).payload(&self.buffer[..])).unwrap(); 70 | self.counter.fetch_add(1, Ordering::SeqCst); 71 | self.producer.poll(std::time::Duration::from_millis(0)); 72 | self.buffer.clear(); 73 | } 74 | } 75 | 76 | impl Drop for EventProducer { 77 | fn drop(&mut self) { 78 | while self.counter.load(Ordering::SeqCst) > 0 { 79 | self.producer.poll(std::time::Duration::from_millis(10)); 80 | } 81 | } 82 | } 83 | 84 | /// A Wrapper for `R: Read` implementing `EventIterator`. 85 | pub struct EventConsumer { 86 | consumer: BaseConsumer, 87 | buffer: Vec, 88 | phant: ::std::marker::PhantomData<(T,D)>, 89 | } 90 | 91 | impl EventConsumer { 92 | /// Allocates a new `EventReader` wrapping a supplied reader. 93 | pub fn new(config: ClientConfig, topic: String) -> Self { 94 | println!("allocating consumer for topic {:?}", topic); 95 | let consumer : BaseConsumer = config.create().expect("Couldn't create consumer"); 96 | consumer.subscribe(&[&topic]).expect("Failed to subscribe to topic"); 97 | EventConsumer { 98 | consumer: consumer, 99 | buffer: Vec::new(), 100 | phant: ::std::marker::PhantomData, 101 | } 102 | } 103 | } 104 | 105 | impl EventIterator for EventConsumer { 106 | fn next(&mut self) -> Option<&Event> { 107 | if let Some(result) = self.consumer.poll(std::time::Duration::from_millis(0)) { 108 | match result { 109 | Ok(message) => { 110 | self.buffer.clear(); 111 | self.buffer.extend_from_slice(message.payload().unwrap()); 112 | Some(unsafe { ::abomonation::decode::>(&mut self.buffer[..]).unwrap().0 }) 113 | }, 114 | Err(err) => { 115 | println!("KafkaConsumer error: {:?}", err); 116 | None 117 | }, 118 | } 119 | } 120 | else { None } 121 | } 122 | } 123 | -------------------------------------------------------------------------------- /logging/.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /.vscode 3 | Cargo.lock 4 | -------------------------------------------------------------------------------- /logging/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "timely_logging" 3 | version = "0.10.0" 4 | authors = ["Frank McSherry "] 5 | edition = "2018" 6 | 7 | description = "Common timely logging infrastructure" 8 | 9 | documentation = "https://docs.rs/timely/" 10 | homepage = "https://github.com/TimelyDataflow/timely-dataflow" 11 | repository = "https://github.com/TimelyDataflow/timely-dataflow.git" 12 | keywords = ["timely", "dataflow", "logging"] 13 | license = "MIT" -------------------------------------------------------------------------------- /mdbook/.gitignore: -------------------------------------------------------------------------------- 1 | book 2 | -------------------------------------------------------------------------------- /mdbook/src/SUMMARY.md: -------------------------------------------------------------------------------- 1 | # Summary 2 | 3 | [Timely Dataflow](./introduction.md) 4 | 5 | - [Motivation](./chapter_0/chapter_0.md) 6 | - [A Simplest Example](./chapter_0/chapter_0_0.md) 7 | - [A Simple Example](./chapter_0/chapter_0_1.md) 8 | - [When to use Timely Dataflow](./chapter_0/chapter_0_2.md) 9 | - [When not to use Timely Dataflow](./chapter_0/chapter_0_3.md) 10 | 11 | - [Core Concepts](./chapter_1/chapter_1.md) 12 | - [Dataflow](./chapter_1/chapter_1_1.md) 13 | - [Timestamps](./chapter_1/chapter_1_2.md) 14 | - [Progress](./chapter_1/chapter_1_3.md) 15 | 16 | - [Building Timely Dataflows](./chapter_2/chapter_2.md) 17 | - [Creating Inputs](./chapter_2/chapter_2_1.md) 18 | - [Observing Outputs](./chapter_2/chapter_2_2.md) 19 | - [Adding Operators](./chapter_2/chapter_2_3.md) 20 | - [Creating Operators](./chapter_2/chapter_2_4.md) 21 | - [A Worked Example](./chapter_2/chapter_2_5.md) 22 | 23 | - [Running Timely Dataflows](./chapter_3/chapter_3.md) 24 | - [Providing Input](./chapter_3/chapter_3_1.md) 25 | - [Monitoring Probes](./chapter_3/chapter_3_2.md) 26 | - [Operator Execution](./chapter_3/chapter_3_3.md) 27 | - [Extending Dataflows](./chapter_3/chapter_3_4.md) 28 | 29 | - [Advanced Timely Dataflow](./chapter_4/chapter_4.md) 30 | - [Scopes](./chapter_4/chapter_4_1.md) 31 | - [Iteration](./chapter_4/chapter_4_2.md) 32 | - [Flow Control](./chapter_4/chapter_4_3.md) 33 | - [Capture and Replay](./chapter_4/chapter_4_4.md) 34 | - [Custom Datatypes](./chapter_4/chapter_4_5.md) 35 | 36 | - [Internals](./chapter_5/chapter_5.md) 37 | - [Communication](./chapter_5/chapter_5_1.md) 38 | - [Progress Tracking](./chapter_5/chapter_5_2.md) 39 | -------------------------------------------------------------------------------- /mdbook/src/chapter_0/chapter_0.md: -------------------------------------------------------------------------------- 1 | # Motivation 2 | 3 | Let's start with some motivation: what can you do with timely dataflow, and when should you be excited to use it as opposed to other programming frameworks? Is timely dataflow great for everything, or is it only great for a few things? Is it great for anything? We will try and clarify these questions in this section. 4 | -------------------------------------------------------------------------------- /mdbook/src/chapter_0/chapter_0_0.md: -------------------------------------------------------------------------------- 1 | ## A simplest example 2 | 3 | Let's start with what may be the simplest non-trivial timely dataflow program. 4 | 5 | ```rust 6 | extern crate timely; 7 | 8 | use timely::dataflow::operators::{ToStream, Inspect}; 9 | 10 | fn main() { 11 | timely::example(|scope| { 12 | (0..10).to_stream(scope) 13 | .inspect(|x| println!("seen: {:?}", x)); 14 | }); 15 | } 16 | ``` 17 | 18 | This program gives us a bit of a flavor for what a timely dataflow program might look like, including a bit of what Rust looks like, without getting too bogged down in weird stream processing details. Not to worry; we will do that in just a moment! 19 | 20 | If we run the program up above, we see it print out the numbers zero through nine. 21 | 22 | ```ignore 23 | Echidnatron% cargo run --example simple 24 | Finished dev [unoptimized + debuginfo] target(s) in 0.05s 25 | Running `target/debug/examples/simple` 26 | seen: 0 27 | seen: 1 28 | seen: 2 29 | seen: 3 30 | seen: 4 31 | seen: 5 32 | seen: 6 33 | seen: 7 34 | seen: 8 35 | seen: 9 36 | Echidnatron% 37 | ``` 38 | 39 | This isn't very different from a Rust program that would do this much more simply, namely the program 40 | 41 | ```rust 42 | fn main() { 43 | (0..10).for_each(|x| println!("seen: {:?}", x)); 44 | } 45 | ``` 46 | 47 | Why would we want to make our life so complicated? The main reason is that we can make our program *reactive*, so that we can run it without knowing ahead of time the data we will use, and it will respond as we produce new data. -------------------------------------------------------------------------------- /mdbook/src/chapter_0/chapter_0_2.md: -------------------------------------------------------------------------------- 1 | # When to use Timely Dataflow 2 | 3 | Timely dataflow may be a different programming model than you are used to, but if you can adapt your program to it there are several benefits. 4 | 5 | * **Data Parallelism**: The operators in timely dataflow are largely "data-parallel", meaning they can operate on independent parts of the data concurrently. This allows the underlying system to distribute timely dataflow computations across multiple parallel workers. These can be threads on your computer, or even threads across computers in a cluster you have access to. This distribution typically improves the throughput of the system, and lets you scale to larger problems with access to more resources (computation, communication, and memory). 6 | 7 | * **Streaming Data**: The core data type in timely dataflow is a *stream* of data, an unbounded collection of data not all of which is available right now, but which instead arrives as the computation proceeds. Streams are a helpful generalization of static data sets, which are assumed available at the start of the computation. By expressing your program as a computation on streams, you've explained both how it should respond to static input data sets (feed all the data in at once) but also how it should react to new data that might arrive later on. 8 | 9 | * **Expressivity**: Timely dataflow's main addition over traditional stream processors is its ability to express higher-level control constructs, like iteration. This moves stream computations from the limitations of straight line code to the world of *algorithms*. Many of the advantages of timely dataflow computations come from our ability to express a more intelligent algorithm than the alternative systems, which can only express more primitive computations. 10 | 11 | There are many domains where streaming and scalability are important, and I'm not sure I can name them all. If you would like to build a scalable monitoring application for a service you run, timely dataflow can express this. If you would like to work with big data computations processing more data than your computer can load into memory, timely dataflow streams can represent this efficiently. If you would like to build an incremental iterative computation over massive data (e.g. matrices, large graphs, text corpora), time dataflow has done these things. 12 | 13 | At the same time, dataflow computation is also another way of thinking about your program. Much like Rust causes you to think a bit harder about program structure, timely dataflow helps you tease out some structure to your program that results in a more effective computation. Even when writing something like `grep`, a program that scans lines of text looking for patterns, by stating your program as a dataflow computation its implementation immediately scales out to multiple threads, and even across multiple computers. 14 | 15 | ## Generality 16 | 17 | Is timely dataflow always applicable? The intent of this research project is to remove layers of abstraction fat that prevent you from expressing anything your computer can do efficiently in parallel. 18 | 19 | Under the covers, your computer (the one on which you are reading this text) is a dataflow processor. When your computer *reads memory* it doesn't actually wander off to find the memory, it introduces a read request into your memory controller, an independent component that will eventually return with the associated cache line. Your computer then gets back to work on whatever it was doing, hoping the responses from the controller return in a timely fashion. 20 | 21 | Academically, I treat "my computer can do this, but timely dataflow cannot" as a bug. There are degrees, of course, and timely datalow isn't on par with the processor's custom hardware designed to handle low level requests efficiently, but *algorithmically*, the goal is that anything you can do efficiently with a computer you should be able to express in timely dataflow. 22 | -------------------------------------------------------------------------------- /mdbook/src/chapter_0/chapter_0_3.md: -------------------------------------------------------------------------------- 1 | # When not to use Timely Dataflow 2 | 3 | There are several reasons not to use timely dataflow, though many of them are *friction* about how your problem is probably expressed, rather than fundamental technical limitations. There are fundamental technical limitations too, of course. 4 | 5 | I've collected a few examples here, but the list may grow with input and feedback. 6 | 7 | --- 8 | 9 | Timely dataflow is a *dataflow* system, and this means that at its core it likes to move data around. This makes life complicated when you would prefer not to move data, and instead move things like pointers and references to data that otherwise stays put. 10 | 11 | For example, sorting a slice of data is a fundamental task and one that parallelizes. But, the task of sorting is traditionally viewed as transforming the data in a supplied slice, rather than sending the data to multiple workers and then announcing that it got sorted. The data really does need to end up in one place, one single pre-existing memory allocation, and timely dataflow is not great at problems that cannot be recast as the movement of data. 12 | 13 | One could re-imagine the sorting process as moving data around, and indeed this is what happens when large clusters need to be brought to bear on such a task, but that doesn't help you at all if what you needed was to sort your single allocation. A library like [Rayon](https://github.com/nikomatsakis/rayon) would almost surely be better suited to the task. 14 | 15 | --- 16 | 17 | Dataflow systems are also fundamentally about breaking apart the execution of your program into independently operating parts. However, many programs are correct only because some things happen *before* or *after* other things. A classic example is [depth-first search](https://en.wikipedia.org/wiki/Depth-first_search) in a graph: although there is lots of work to do on small bits of data, it is crucial that the exploration of nodes reachable along a graph edge complete before the exploration of nodes reachable along the next graph edge. 18 | 19 | Although there is plenty of active research on transforming algorithms from sequential to parallel, if you aren't clear on how to express your program as a dataflow program then timely dataflow may not be a great fit. At the very least, the first step would be "fundamentally re-imagine your program", which can be a fine thing to do, but is perhaps not something you would have to do with your traditional program. 20 | 21 | --- 22 | 23 | Timely dataflow is in a bit of a weird space between language library and runtime system. This means that it doesn't quite have the stability guarantees a library might have (when you call `data.sort()` you don't think about "what if it fails?"), nor does it have the surrounding infrastructure of a [DryadLINQ](https://www.microsoft.com/en-us/research/project/dryadlinq/) or [Spark](https://spark.apache.org) style of experience. Part of this burden is simply passed to you, and this may be intolerable depending on your goals for your program. 24 | -------------------------------------------------------------------------------- /mdbook/src/chapter_1/chapter_1.md: -------------------------------------------------------------------------------- 1 | # Chapter 1: Core Concepts 2 | 3 | Timely dataflow relies on two fundamental concepts: **timestamps** and **dataflow**, which together lead to the concept of **progress**. We will want to break down these concepts because they play a fundamental role in understanding how timely dataflow programs are structured. 4 | 5 | ## Dataflow 6 | 7 | Dataflow programming is fundamentally about describing your program as independent components, each of which operate in response to the availability of input data, as well as describing the connections between these components. 8 | 9 | The most important part of dataflow programming is the *independence* of the components. When you write a dataflow program, you provide the computer with flexibility in how it executes your program. Rather than insisting on a specific sequence of instructions the computer should follow, the computer can work on each of the components as it sees fit, perhaps even sharing the work with other computers. 10 | 11 | ## Timestamps 12 | 13 | While we want to enjoy the benefits of dataflow programming, we still need to understand whether and how our computation progresses. In traditional imperative programming we could reason that because instructions happen in some order, then once we reach a certain point all work (of a certain type) must be done. Instead, we will tag the data that move through our dataflow with *timestamps*, indicating (roughly) when they would have happened in a sequential execution. 14 | 15 | Timestamps play at least two roles in timely dataflow: they allow dataflow components to make sense of the otherwise unordered inputs they see ("ah, I received the data in *this* order, but I should behave as if it arrived in *this* order"), and they allow the user (and others) to reason about whether they have seen all of the data with a certain timestamp. 16 | 17 | Timestamps allow us to introduce sequential structure into our program, without requiring actual sequential execution. 18 | 19 | ## Progress 20 | 21 | In a traditional imperative program, if we want to return the maximum of a set of numbers, we just scan all the numbers and return the maximum. We don't have to worry about whether we've considered *all* of the numbers yet, because the program makes sure not to provide an answer until it has consulted each number. 22 | 23 | This simple task is much harder in a dataflow setting, where numbers arrive as input to a component that is tracking the maximum. Before releasing a number as output, the component must know if it has seen everything, as one more value could change its answer. But strictly speaking, nothing we've said so far about dataflow or timestamps provide any information about whether more data might arrive. 24 | 25 | If we combine dataflow program structure with timestamped data in such a way that as data move along the dataflow their timestamps only increase, we are able to reason about the *progress* of our computation. More specifically, at any component in the dataflow, we can reason about which timestamps we may yet see in the future. Timestamps that are no longer possible are considered "passed", and components can react to this information as they see fit. 26 | 27 | Continual information about the progress of a computation is the only basis of coordination in timely dataflow, and is the lightest touch we could think of. 28 | -------------------------------------------------------------------------------- /mdbook/src/chapter_1/chapter_1_2.md: -------------------------------------------------------------------------------- 1 | # Logical Timestamps 2 | 3 | When dataflow programs move data around arbitrarily, it becomes hard to correlate the produced outputs with the supplied inputs. If we supply a stream of bank transactions as input, and the output is a stream of bank balances, how can we know which input transactions are reflected in which output balances? 4 | 5 | The standard approach to this problem is to install *timestamps* on the data. Each records gets a logical timestamp associated with it that indicates *when* it should be thought to happen. This is not necessarily "when" in terms of the date, time, or specific nanosecond the record was emitted; a timestamp could simply be a sequence number identifying a batch of input records. Or, and we will get in to the terrifying details later, it could be much more complicated than this. 6 | 7 | Timestamps are what allow us to correlate inputs and outputs. When we introduce records with some logical timestamp, unless our dataflow computation changes the timestamps, we expect to see corresponding outputs with that same timestamp. 8 | 9 | ## An example 10 | 11 | Remember from the dataflow section how when we remove the coordination from our `examples/hello.rs` program, the output was produced in some horrible order? In fact, each of those records had a timestamp associated with it that would reveal the correct order; we just weren't printing the timestamp because `inspect` doesn't have access to it. 12 | 13 | Let's change the program to print out the timestamp with each record. This shouldn't be very thrilling output, because the timestamp is exactly the same as the number itself, but that didn't have to be the case. We are just going to replace the line 14 | 15 | ```rust,ignore 16 | .inspect(move |x| println!("worker {}:\thello {}", index, x)) 17 | ``` 18 | 19 | with a slightly more complicated operator, `inspect_batch`. 20 | 21 | ```rust,ignore 22 | .inspect_batch(move |t,xs| { 23 | for x in xs.iter() { 24 | println!("worker {}:\thello {} @ {:?}", index, x, t) 25 | } 26 | }) 27 | ``` 28 | 29 | The `inspect_batch` operator gets lower-level access to data in timely dataflow, in particular access to batches of records with the same timestamp. It is intended for diagnosing system-level details, but we can also use it to see what timestamps accompany the data. 30 | 31 | The output we get with two workers is now: 32 | 33 | ```ignore 34 | Echidnatron% cargo run --example hello -- -w2 35 | Finished dev [unoptimized + debuginfo] target(s) in 0.0 secs 36 | Running `target/debug/examples/hello -w2` 37 | worker 1: hello 1 @ (Root, 1) 38 | worker 1: hello 3 @ (Root, 3) 39 | worker 1: hello 5 @ (Root, 5) 40 | worker 0: hello 0 @ (Root, 0) 41 | worker 0: hello 2 @ (Root, 2) 42 | worker 0: hello 4 @ (Root, 4) 43 | worker 0: hello 6 @ (Root, 6) 44 | worker 0: hello 8 @ (Root, 8) 45 | worker 1: hello 7 @ (Root, 7) 46 | worker 1: hello 9 @ (Root, 9) 47 | Echidnatron% 48 | ``` 49 | 50 | The timestamps are the `(Root, i)` things for various values of `i`. These happen to correspond to the data themselves, but had we provided random input data rather than `i` itself we would still be able to make sense of the output and put it back "in order". 51 | 52 | ## Timestamps for dataflow operators 53 | 54 | Timestamps are not only helpful for dataflow users, but for the operators themselves. With time we will start to write more interesting dataflow operators, and it may be important for them to understand which records should be thought to come before others. 55 | 56 | Imagine, for example, a dataflow operator whose job is to report the "sum so far", where "so far" should be with respect to the timestamp (as opposed to whatever arbitary order the operator receives the records). Such an operator can't simply take its input records, add them to a total, and produce the result. The input records may no longer be ordered by timestamp, and the produced summations may not reflect any partial sum of the input. Instead, the operator needs to look at the timestamps on the records, and incorporate the numbers in order of their timestamps. 57 | 58 | Of course, such an operator works great as long as it expects exactly one record for each timestamp. Things get harder for it if it might receive multiple records at each timestamp, or perhaps none. To address this, the underlying system will have to help the operator reason about the progress of its input, up next. 59 | -------------------------------------------------------------------------------- /mdbook/src/chapter_1/chapter_1_3.md: -------------------------------------------------------------------------------- 1 | # Tracking Progress 2 | 3 | Both dataflow and timestamps are valuable in their own right, but when we bring them together we get something even better. We get the ability to reason about the flow of timestamps through our computation, and we recover the ability to inform each dataflow component about how much of its input data it has seen. 4 | 5 | Let's recall that bit of code we commented out from `examples/hello.rs`, which had to do with consulting something named `probe`. 6 | 7 | ```rust 8 | extern crate timely; 9 | 10 | use timely::dataflow::InputHandle; 11 | use timely::dataflow::operators::{Input, Exchange, Inspect, Probe}; 12 | 13 | fn main() { 14 | // initializes and runs a timely dataflow. 15 | timely::execute_from_args(std::env::args(), |worker| { 16 | 17 | let index = worker.index(); 18 | let mut input = InputHandle::new(); 19 | 20 | // create a new input, exchange data, and inspect its output 21 | let probe = worker.dataflow(|scope| 22 | scope.input_from(&mut input) 23 | .exchange(|x| *x) 24 | .inspect(move |x| println!("worker {}:\thello {}", index, x)) 25 | .probe() 26 | ); 27 | 28 | // introduce data and watch! 29 | for round in 0..10 { 30 | if worker.index() == 0 { 31 | input.send(round); 32 | } 33 | input.advance_to(round + 1); 34 | worker.step_while(|| probe.less_than(input.time())); 35 | } 36 | }).unwrap(); 37 | } 38 | ``` 39 | 40 | We'll put the whole program up here, but there are really just two lines that deal with progress tracking: 41 | 42 | ```rust,ignore 43 | input.advance_to(round + 1); 44 | worker.step_while(|| probe.less_than(input.time())); 45 | ``` 46 | 47 | Let's talk about each of them. 48 | 49 | ## Input capabilities 50 | 51 | The `input` structure is how we provide data to a timely dataflow computation, and it has a timestamp associated with it. Initially this timestamp is the default value, usually something like `0` for integers. Whatever timestamp `input` has, it can introduce data with that timestamp or greater. We can advance this timestamp, via the `advance_to` method, which restricts the timestamps we can use to those greater or equal to whatever timestamp is supplied as the argument. 52 | 53 | The `advance_to` method is a big deal. This is the moment in the computation where our program reveals to the system, and through the system to all other dataflow workers, that we might soon be able to announce a timestamp as complete. There may still be records in flight bearing that timestamp, but as they are retired the system can finally report that progress has been made. 54 | 55 | ## Output possibilities 56 | 57 | The `probe` structure is how we learn about the possibility of timestamped data at some point in the dataflow graph. We can, at any point, consult a probe with the `less_than` method and ask whether it is still possible that we might see a time less than the argument at that point in the dataflow graph. There is also a `less_equal` method, if you prefer that. 58 | 59 | Putting a probe after the `inspect` operator, which passes through all data it receives as input only after invoking its method, tells us whether we should expect to see the method associated with `inspect` fire again for a given timestamp. If we are told we won't see any more messages with timestamp `t` after the `inspect`, then the `inspect` won't see any either. 60 | 61 | The `less_than` and `less_equal` methods are the only place where we learn about the state of the rest of the system. These methods are non-blocking; they always return immediately with either a "yes, you might see such a timestamp" or a "no, you will not see such a timestamp". 62 | 63 | ## Responding to progress information 64 | 65 | Progress information is relatively passive. We get to observe what happens in the rest of the system, and perhaps change our behavior based on the amount of progress. We do not get to tell the system what to do next, we just get to see what has happened since last we checked. 66 | 67 | This passive approach to coordination allows the system to operate with minimal overhead. Workers exchange both data and progress information. If workers want to wait for further progress before introducing more data they see they are welcome to do so, but they can also go and work on a different part of the dataflow graph as well. 68 | 69 | Progress information provides a relatively unopinionated view of coordination. Workers are welcome to impose a more synchronous discipline using progress information, perhaps proceeding in sequence through operators by consulting probes installed after each of them, but they are not required to do so. Synchronization is possible, but it becomes a choice made by the workers themselves, rather than imposed on them by the system. 70 | -------------------------------------------------------------------------------- /mdbook/src/chapter_2/chapter_2.md: -------------------------------------------------------------------------------- 1 | # Building Timely Dataflows 2 | 3 | Let's talk about how to create timely dataflows. 4 | 5 | This section will be a bit of a tour through the dataflow construction process, ignoring for the moment details about the interesting ways in which you can get data in to and out of your dataflow; those will show up in the "Running Timely Dataflows" section. For now we are going to work with examples with fixed input data and no interactivity to speak of, focusing on what we can cause to happen to that data. 6 | 7 | Here is a relatively simple example, taken from `timely/examples/simple.rs`, that turns the numbers zero through nine into a stream, and then feeds them through an `inspect` operator printing them to the screen. 8 | 9 | ```rust 10 | extern crate timely; 11 | 12 | use timely::dataflow::operators::{ToStream, Inspect}; 13 | 14 | fn main() { 15 | timely::example(|scope| { 16 | (0..10).to_stream(scope) 17 | .inspect(|x| println!("seen: {:?}", x)); 18 | }); 19 | } 20 | ``` 21 | 22 | We are going to develop out this example, showing off both the built-in operators as well as timely's generic operator construction features. 23 | 24 | --- 25 | 26 | **NOTE**: Timely very much assumes that you are going to build the same dataflow on each worker. You don't literally have to, in that you could build a dataflow from user input, or with a random number generator, things like that. Please don't! It will not be a good use of your time. 27 | -------------------------------------------------------------------------------- /mdbook/src/chapter_2/chapter_2_1.md: -------------------------------------------------------------------------------- 1 | # Creating Inputs 2 | 3 | Let's start with the first thing we'll want for a dataflow computation: a source of data. 4 | 5 | Almost all operators in timely can only be defined from a source of data, with a few exceptions. One of these exceptions is the `to_stream` operator, which is defined for various types and which takes a `scope` as an argument and produces a stream in that scope. Our `InputHandle` type from previous examples has a `to_stream` method, as well as any type that can be turned into an iterator (which we used in the preceding example). 6 | 7 | For example, we can create a new dataflow with one interactive input and one static input: 8 | 9 | ```rust 10 | extern crate timely; 11 | 12 | use timely::dataflow::InputHandle; 13 | use timely::dataflow::operators::ToStream; 14 | 15 | fn main() { 16 | // initializes and runs a timely dataflow. 17 | timely::execute_from_args(std::env::args(), |worker| { 18 | 19 | let mut input = InputHandle::<(), String>::new(); 20 | 21 | // define a new dataflow 22 | worker.dataflow(|scope| { 23 | 24 | let stream1 = input.to_stream(scope); 25 | let stream2 = (0 .. 9).to_stream(scope); 26 | 27 | }); 28 | 29 | }).unwrap(); 30 | } 31 | ``` 32 | 33 | There will be more to do to get data in to `input`, and we aren't going to worry about that at the moment. But, now you know two of the places you can get data from! 34 | 35 | ## Other sources 36 | 37 | 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`). 38 | -------------------------------------------------------------------------------- /mdbook/src/chapter_2/chapter_2_2.md: -------------------------------------------------------------------------------- 1 | # Observing Outputs 2 | 3 | Having constructed a minimal streaming computation, we might like to take a peek at the output. There are a few ways to do this, but the simplest by far is the `inspect` operator. 4 | 5 | The `inspect` operator is called with a closure, and it ensures that the closure is run on each record that passes through the operator. This closure can do just about anything, from printing to the screen or writing to a file. 6 | 7 | ```rust 8 | extern crate timely; 9 | 10 | use timely::dataflow::operators::{ToStream, Inspect}; 11 | 12 | fn main() { 13 | timely::execute_from_args(std::env::args(), |worker| { 14 | worker.dataflow::<(),_,_>(|scope| { 15 | (0 .. 9) 16 | .to_stream(scope) 17 | .inspect(|x| println!("hello: {}", x)); 18 | }); 19 | }).unwrap(); 20 | } 21 | ``` 22 | 23 | This simple example turns the sequence zero through nine into a stream and then prints the results to the screen. 24 | 25 | ## Inspecting Batches 26 | 27 | The `inspect` operator has a big sibling, `inspect_batch`, whose closure gets access to whole batches of records at a time, just like the underlying operator. More precisely, `inspect_batch` takes a closure of two parameters: first, the timestamp of a batch, and second a reference to the batch itself. The `inspect_batch` operator can be especially helpful if you want to process the outputs more efficiently. 28 | 29 | ```rust 30 | extern crate timely; 31 | 32 | use timely::dataflow::operators::{ToStream, Inspect}; 33 | 34 | fn main() { 35 | timely::execute_from_args(std::env::args(), |worker| { 36 | worker.dataflow::<(),_,_>(|scope| { 37 | (0 .. 10) 38 | .to_stream(scope) 39 | .inspect_batch(|t, xs| println!("hello: {:?} @ {:?}", xs, t)); 40 | }); 41 | }).unwrap(); 42 | } 43 | ``` 44 | 45 | ## Capturing Streams 46 | 47 | The `Capture` trait provides a mechanism for exfiltrating a stream from a dataflow, into information that can be replayed in other dataflows. The trait is pretty general, and can even capture a stream to a binary writer that can be read back from to reconstruct the stream (see `examples/capture_send.rs` and `examples/capture_recv.rs`). 48 | 49 | The simplest form of capture is the `capture()` method, which turns the stream into a shared queue of "events", which are the sequence of events the operator is exposed to: data arriving and notification of progress through the input stream. The `capture` method is used in many of timely's documentation tests, to extract a stream and verify that it is correct. 50 | 51 | Consider the documentation test for the `ToStream` trait: 52 | 53 | ```rust 54 | extern crate timely; 55 | 56 | use timely::dataflow::operators::{ToStream, Capture}; 57 | use timely::dataflow::operators::capture::Extract; 58 | 59 | fn main() { 60 | let (data1, data2) = timely::example(|scope| { 61 | let data1 = (0..3).to_stream(scope).capture(); 62 | let data2 = vec![0,1,2].to_stream(scope).capture(); 63 | (data1, data2) 64 | }); 65 | 66 | assert_eq!(data1.extract(), data2.extract()); 67 | } 68 | ``` 69 | 70 | Here the two `capture` methods each return the receive side of one of Rust's threadsafe channels. The data moving along the channel have a type `capture::Event` which you would need to read about, but which your main thread can drain out of the channel and process as it sees fit. 71 | -------------------------------------------------------------------------------- /mdbook/src/chapter_3/chapter_3.md: -------------------------------------------------------------------------------- 1 | # Running Timely Dataflows 2 | 3 | In this section we will look at driving a timely dataflow computation. 4 | 5 | With a dataflow graph defined, how do we interactively supply data to the computation, and how do we understand what the computation has actually done given that we are not ourselves doing it? These are good questions, and the dataflow execution model is indeed a bit of a departure from how most folks first experience programming. 6 | 7 | The first thing to understand about timely dataflow is that *we are programming the worker threads*. Part of this program is asking the worker to build up a dataflow graph; we did that when we created an `InputHandle` and when we called `dataflow` followed by some `filter` and `map` and `probe` commands. But another part is where we actually start feeding the dataflow graph, advancing the inputs, and letting the worker give each of the operators a chance to run. 8 | 9 | ```rust,ignore 10 | for round in 0..10 { 11 | input.send(round); 12 | input.advance_to(round + 1); 13 | while probe.less_than(input.time()) { 14 | worker.step(); 15 | } 16 | } 17 | ``` 18 | 19 | This is the loop that we've seen in several examples. It looks fairly simple, but this is what actually causes work to happen. We do send data and advance the input, but we also call `worker.step()`, and this is where the actual timely dataflow computation happens. Until you call this, all the data are just building up in queues. 20 | 21 | In this section, we'll look at these moving parts in more detail. 22 | -------------------------------------------------------------------------------- /mdbook/src/chapter_3/chapter_3_1.md: -------------------------------------------------------------------------------- 1 | # Providing Input 2 | 3 | The first thing we often see is `input.send` with some data. This moves the supplied data from the current scope into a queue shared with the input dataflow operator. As this queue starts to fill, perhaps due to you calling `send` a lot, it moves the data along to its intended recipients. This probably means input queues of other operators, but it may mean serialization and network transmission. 4 | 5 | You can call `send` as much as you like, and the `InputHandle` implementation will keep moving the data along. The worst that is going to happen is depositing data in shared queues and perhaps some serialization; the call to `send` will not block, and it should not capture your thread of execution to do any particularly expensive work. 6 | 7 | However, since we are writing the worker code, you may want to take a break from `send` every now and again and let some of the operators run (in just a moment). Part of efficient streaming computation is keeping the data moving, and building up only relatively small buffers before giving the operators a chance to run. 8 | 9 | ## Controlling capabilities 10 | 11 | The second thing we often see is `input.advance_to` with a time. This is an exciting moment where the input announces that it will no longer send data timestamped with anything not greater or equal to its argument. This is big news for the rest of the system, as any operator waiting on the timestamp you previously held can now get to work (or, once all the messages you sent have drained, it can get to work). 12 | 13 | It is a logic error to call `advance_to` with a time that is not greater or equal to the current time, which you can read out with `input.time`. Timely will check this for you and panic if you screw it up. It is a bit like accessing an array out of bounds: you can check ahead of time if you are about to screw up, but you went and did it anyhow. 14 | 15 | Finally, you might be interested to call `input.close`. This method consumes the input and thereby prevents you from sending any more data. This information is *very* exciting to the system, which can now tell dataflow operators that they won't be hearing much of anything from you any more. 16 | 17 | **TIP**: It is very important to keep moving your inputs along if you want your dataflow graph to make progress. One of the most common classes of errors is forgetting to advance an `InputHandle`, and then waiting and waiting and waiting for the cumulative count of records (or whatever) to come out the other end. Timely really wants you to participate and be clear about what you will and will not do in the future. 18 | 19 | At the same time, timely's progress tracking does work proportional to the number of timestamps you introduce. If you use a new timestamp for every record, timely will flush its buffers a lot, get very angry with you, and probably fall over. To the extent that you can batch inputs, sending many with the same timestamp, the better. 20 | -------------------------------------------------------------------------------- /mdbook/src/chapter_3/chapter_3_2.md: -------------------------------------------------------------------------------- 1 | # Monitoring Probes 2 | 3 | On the flip side of inputs we have probes. Probes aren't *outputs* per se, but rather ways for you to monitor progress. They report on the possible timestamps remaining at certain places in the dataflow graph (wherever you attach them). 4 | 5 | The easiest way to create a `ProbeHandle` is by calling `.probe()` on a stream. This attaches a probe to a point in the dataflow, and when you inspect the probe (in just a moment) you'll learn about what might happen at that point. 6 | 7 | You can also create a `ProbeHandle` directly with `ProbeHandle::new()`. Such a probe handle is not very interesting yet, but you can attach a probe handle by calling `.probe_with(&mut handle)` on a stream. This has the cute benefit that you can attach one probe to multiple parts of the dataflow graph, and it will report on the union of their times. If you would like to watch multiple outputs, you could call `.probe()` multiple times, or attach one common handle to each with multiple calls to `.probe_with()`. Both are reasonable, depending on whether you need to distinguish between the multiple locations. 8 | 9 | A probe handle monitors information that timely provides about the availability of timestamps. You can think of it as holding on to a `Vec