├── .gitignore ├── CONTRIBUTORS ├── LICENSE ├── README.md ├── connect ├── .gitignore ├── Cargo.toml ├── release.toml └── src │ ├── lib.rs │ └── receive │ ├── connect.rs │ ├── mod.rs │ └── replaywithshutdown.rs ├── examples └── pagerank.png └── tdiag ├── .gitignore ├── Cargo.toml ├── release.toml └── src ├── commands ├── arrangements.rs ├── graph.rs ├── graph │ └── dataflow-graph.html ├── mod.rs └── profile.rs ├── lib.rs └── main.rs /.gitignore: -------------------------------------------------------------------------------- 1 | # Generated by Cargo 2 | # will have compiled files and executables 3 | /target/ 4 | 5 | # Remove Cargo.lock from gitignore if creating an executable, leave it for libraries 6 | # More information here https://doc.rust-lang.org/cargo/guide/cargo-toml-vs-cargo-lock.html 7 | Cargo.lock 8 | 9 | # These are backup files generated by rustfmt 10 | **/*.rs.bk 11 | -------------------------------------------------------------------------------- /CONTRIBUTORS: -------------------------------------------------------------------------------- 1 | Andrea Lattuada 2 | Nikolas Göbel 3 | Nazerke Seidan 4 | Frank McSherry 5 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2019 Timely Dataflow Diagnostics contributors 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 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Timely Diagnostics 2 | 3 | Diagnostic tools for [timely 4 | dataflow](https://github.com/TimelyDataflow/timely-dataflow) 5 | computations. Timely dataflows are data-parallel and scale from single 6 | threaded execution on your laptop to distributed execution across 7 | clusters of computers. Each thread of execution is called a *worker*. 8 | 9 | The tools in this repository have the shared goal of providing 10 | insights into timely dataflows of any scale, in order to understand 11 | the structure and resource usage of a dataflow. 12 | 13 | Each timely worker can be instructed to publish low-level event 14 | streams over a TCP socket, by setting the `TIMELY_WORKER_LOG_ADDR` 15 | environment variable. In order to cope with the high volume of 16 | these logging streams the diagnostic tools in this repository 17 | are themselves timely computations that we can scale out. In order to avoid 18 | confusion, we will refer to the workers of the dataflow *that is being 19 | analysed* as the `source peers`. The workers of the diagnostic 20 | computation we will refer to as `inspector 21 | peers`. 22 | 23 | This repository contains a library, `tdiag-connect`, and a command 24 | line interface to the diagnostic tools, `tdiag`. 25 | 26 | `tdiag-connect` (in [`/connect`](./connect)) is a library of utiltities that can 27 | be used by inspector peers to source event streams from source peers. 28 | 29 | `tdiag` (in [/tdiag](./tdiag)) is an unified command line interface to all diagnostic 30 | tools (only one is currently available, more are coming). 31 | 32 | ## Getting Started with `tdiag` 33 | 34 | `tdiag` [![Crates.io](https://img.shields.io/crates/v/tdiag.svg)](https://crates.io/crates/tdiag) is the CLI to all diagnostic tools. Install it via cargo: 35 | 36 | ``` shell 37 | cargo install tdiag 38 | ``` 39 | 40 | All diagnostic computations require you to specify the number of 41 | workers running in the source computation via the `source-peers` 42 | parameter. This is required in order to know when all source event 43 | streams are connected. 44 | 45 | ### `graph` - Visualize the Source Dataflow 46 | 47 | In order to better understand what is happening inside of a dataflow 48 | computation, it can be invaluable to visualize the structure of the 49 | dataflow. Start the `graph` diagnosis: 50 | 51 | ``` shell 52 | tdiag --source-peers 2 graph --out graph.html 53 | ``` 54 | 55 | You should be presented with a notice, informing you that `tdiag` is 56 | waiting for as many connections as specified via `--source-peers` (two 57 | in this case). 58 | 59 | In a separate shell, start your source computation. In this case, we 60 | will analyse the [Timely PageRank 61 | example](https://github.com/TimelyDataflow/timely-dataflow/blob/master/timely/examples/pagerank.rs). From 62 | inside the `timely-dataflow/timely` sub-directory, run: 63 | 64 | ``` shell 65 | env TIMELY_WORKER_LOG_ADDR="127.0.0.1:51317" cargo run --example pagerank 1000 1000000 -w 2 66 | ``` 67 | 68 | Most importantly, `env TIMELY_WORKER_LOG_ADDR="127.0.0.1:51317"` will 69 | cause the source workers to connect to our diagnostic computation. The 70 | `-w` parameter specifies the number of workers we want to run the 71 | PageRank example with. Whatever we specify here therefore has to match 72 | the `--source-peers` parameter we used when starting `tdiag`. 73 | 74 | Once the computation is running, head back to the diagnostic shell, 75 | where you should now see something like the following: 76 | 77 | ``` shell 78 | $ tdiag --source-peers 2 graph --out graph.html 79 | 80 | Listening for 2 connections on 127.0.0.1:51317 81 | Trace sources connected 82 | Press enter to generate graph (this will crash the source computation if it hasn't terminated). 83 | ``` 84 | 85 | At any point, press enter as instructed. This will produce a fully 86 | self-contained HTML file at the path specified via `--out` 87 | (`graph.html` in this example). Open that file in any modern browser 88 | and you should see a rendering of the dataflow graph at the time you 89 | pressed enter. For the PageRank computation, the rendering should look 90 | similar to the following: 91 | 92 | ![PageRank Graph](./examples/pagerank.png) 93 | 94 | You can use your mouse or touchpad to move the graph around, and to 95 | zoom in and out. 96 | 97 | ### `profile` - Profile the Source Dataflow 98 | 99 | The `profile` subcommand reports aggregate runtime for each scope/operator. 100 | 101 | ```shell 102 | tdiag --source-peers 2 profile 103 | ``` 104 | 105 | You should be presented with a notice informing you that `tdiag` is 106 | waiting for as many connections as specified via `--source-peers` (two 107 | in this case). 108 | 109 | In a separate shell, start your source computation. In this case, we 110 | will analyse the [Timely PageRank 111 | example](https://github.com/TimelyDataflow/timely-dataflow/blob/master/timely/examples/pagerank.rs). From 112 | inside the `timely-dataflow/timely` sub-directory, run: 113 | 114 | ``` shell 115 | env TIMELY_WORKER_LOG_ADDR="127.0.0.1:51317" cargo run --example pagerank 1000 1000000 -w 2 116 | ``` 117 | 118 | Most importantly, `env TIMELY_WORKER_LOG_ADDR="127.0.0.1:51317"` will 119 | cause the source workers to connect to our diagnostic computation. The 120 | `-w` parameter specifies the number of workers we want to run the 121 | PageRank example with. Whatever we specify here therefore has to match 122 | the `--source-peers` parameter we used when starting `tdiag`. 123 | 124 | Once the computation is running, head back to the diagnostic shell, 125 | where you should now see something like the following: 126 | 127 | ```shell 128 | $ tdiag --source-peers 2 profile 129 | 130 | Listening for 2 connections on 127.0.0.1:51317 131 | Trace sources connected 132 | Press enter to stop collecting profile data (this will crash the source computation if it hasn't terminated). 133 | ``` 134 | 135 | At any point, press enter as instructed. This will produce an aggregate 136 | summary of runtime for each scope/operator. Note that the aggregates for the 137 | scopes (denoted by `[scope]`) include the time of all contained operators. 138 | 139 | ```shell 140 | [scope] Dataflow (id=0, addr=[0]): 1.17870668e-1 s 141 | PageRank (id=3, addr=[0, 3]): 1.17197194e-1 s 142 | Feedback (id=2, addr=[0, 2]): 3.56249e-4 s 143 | Probe (id=6, addr=[0, 4]): 7.86e-6 s 144 | Input (id=1, addr=[0, 1]): 3.408e-6 s 145 | ``` 146 | 147 | ## Diagnosing Differential Dataflows 148 | 149 | The `differential` subcommand groups diagnostic tools that are only 150 | relevant to timely dataflows that make use of [differential 151 | dataflow](https://github.com/TimelyDataflow/differential-dataflow). To 152 | enable Differential logging in your own computation, add the following 153 | snippet to your code: 154 | 155 | ``` rust 156 | if let Ok(addr) = ::std::env::var("DIFFERENTIAL_LOG_ADDR") { 157 | if let Ok(stream) = ::std::net::TcpStream::connect(&addr) { 158 | differential_dataflow::logging::enable(worker, stream); 159 | info!("enabled DIFFERENTIAL logging to {}", addr); 160 | } else { 161 | panic!("Could not connect to differential log address: {:?}", addr); 162 | } 163 | } 164 | ``` 165 | 166 | With this snippet included in your executable, you can use any of the 167 | following tools to analyse differential-specific aspects of your 168 | computation. 169 | 170 | ### `differential arrangements` - Track the Size of Differential Arrangements 171 | 172 | Stateful differential dataflow operators often maintain indexed input 173 | traces called `arrangements`. You will want to understand how these 174 | traces grow (through the accumulation of new inputs) and shrink 175 | (through compaction) in size, as your computation executes. 176 | 177 | ```shell 178 | tdiag --source-peers differential arrangements 179 | ``` 180 | 181 | You should be presented with a notice informing you that `tdiag` is 182 | waiting for as many connections as specified via `--source-peers` (two 183 | in this case). 184 | 185 | In a separate shell, start your source computation. In this case, we 186 | will analyse the [Differential BFS 187 | example](https://github.com/TimelyDataflow/differential-dataflow/blob/master/examples/bfs.rs). From 188 | inside the differential dataflow repository, run: 189 | 190 | ``` shell 191 | export TIMELY_WORKER_LOG_ADDR="127.0.0.1:51317" 192 | export DIFFERENTIAL_LOG_ADDR="127.0.0.1:51318" 193 | 194 | cargo run --example bfs 1000 10000 100 20 false -w 2 195 | ``` 196 | 197 | When analysing differential dataflows (in contrast to pure timely 198 | computations), both `TIMELY_WORKER_LOG_ADDR` and 199 | `DIFFERENTIAL_LOG_ADDR` must be set for the source workers to connect 200 | to our diagnostic computation. The `-w` parameter specifies the number 201 | of workers we want to run the PageRank example with. Whatever we 202 | specify here therefore has to match the `--source-peers` parameter we 203 | used when starting `tdiag`. 204 | 205 | Once the computation is running, head back to the diagnostic shell, 206 | where you should now see something like the following: 207 | 208 | ```shell 209 | $ tdiag --source-peers 2 differential arrangements 210 | 211 | Listening for 2 Timely connections on 127.0.0.1:51317 212 | Listening for 2 Differential connections on 127.0.0.1:51319 213 | Will report every 1000ms 214 | Trace sources connected 215 | 216 | ms Worker Op. Id Name # of tuples 217 | 1000 0 18 Arrange ([0, 4, 6]) 654 218 | 1000 0 20 Arrange ([0, 4, 7]) 5944 219 | 1000 0 28 Arrange ([0, 4, 10]) 3790 220 | 1000 0 30 Reduce ([0, 4, 11]) 654 221 | 1000 1 18 Arrange ([0, 4, 6]) 679 222 | 1000 1 20 Arrange ([0, 4, 7]) 6006 223 | 1000 1 28 Arrange ([0, 4, 10]) 3913 224 | 1000 1 30 Reduce ([0, 4, 11]) 678 225 | 2000 0 18 Arrange ([0, 4, 6]) 654 226 | 2000 0 18 Arrange ([0, 4, 6]) 950 227 | 2000 0 20 Arrange ([0, 4, 7]) 5944 228 | 2000 0 20 Arrange ([0, 4, 7]) 6937 229 | 2000 0 28 Arrange ([0, 4, 10]) 3790 230 | ``` 231 | 232 | Each row of output specifies the time of the measurement, worker and 233 | operator ids, the name of the arrangement and the number of tuples it 234 | maintains. Updated sizes will be reported every second by default, 235 | this can be controlled via the `output-interval` parameter. 236 | 237 | ## The `tdiag-connect` library 238 | 239 | [![Crates.io](https://img.shields.io/crates/v/tdiag-connect.svg)](https://crates.io/crates/tdiag-connect) [![Docs](https://img.shields.io/badge/docs-.rs-blue.svg)](https://docs.rs/tdiag-connect) 240 | 241 | `tdiag-connect` (in [`/connect`](./connect)) is a library of utiltities that can 242 | be used by inspector peers to source event streams from source peers. 243 | 244 | Documentation is at [docs.rs/tdiag-connect](https://docs.rs/tdiag-connect). 245 | -------------------------------------------------------------------------------- /connect/.gitignore: -------------------------------------------------------------------------------- 1 | target/ 2 | Cargo.lock 3 | -------------------------------------------------------------------------------- /connect/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "tdiag-connect" 3 | version = "0.2.1-pre" 4 | authors = ["Andrea Lattuada "] 5 | homepage = "https://github.com/TimelyDataflow/diagnostics" 6 | repository = "https://github.com/TimelyDataflow/diagnostics.git" 7 | description = "Helpers to obtain traces suitable for diagnostics from timely / differential." 8 | license = "MIT" 9 | 10 | edition = "2018" 11 | 12 | [dependencies] 13 | timely = "^0.11" 14 | -------------------------------------------------------------------------------- /connect/release.toml: -------------------------------------------------------------------------------- 1 | upload-doc = false 2 | tag-prefix = "" 3 | pre-release-commit-message = "[tdiag-connect] Release version {{version}}" 4 | pro-release-commit-message = "[tdiag-connect] Development version {{next_version}}" 5 | tag-message = "version {{version}}" 6 | dev-version-ext = "pre" 7 | -------------------------------------------------------------------------------- /connect/src/lib.rs: -------------------------------------------------------------------------------- 1 | //! Helpers to obtain traces suitable for diagnostics from timely / differential. 2 | 3 | /// An error generated by this crate's connect functions. 4 | #[derive(Debug)] 5 | pub enum ConnectError { 6 | IoError(std::io::Error), 7 | Other(String), 8 | } 9 | 10 | impl From for ConnectError { 11 | fn from(e: std::io::Error) -> Self { 12 | ConnectError::IoError(e) 13 | } 14 | } 15 | 16 | pub mod receive; 17 | -------------------------------------------------------------------------------- /connect/src/receive/connect.rs: -------------------------------------------------------------------------------- 1 | use crate::ConnectError; 2 | 3 | use timely::dataflow::operators::capture::EventReader; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | use std::net::{TcpStream, TcpListener, ToSocketAddrs, IpAddr}; 7 | use std::fs::File; 8 | use std::path::PathBuf; 9 | 10 | /// Listens on 127.0.0.1:8000 and opens `source_peers` sockets from the 11 | /// computations we're examining (one socket for every worker on the 12 | /// examined computation). 13 | /// 14 | /// The sockets are wrapped in `Some(_)` because the result is commonly 15 | /// used as a an argument to `make_readers` in this module. 16 | /// 17 | /// The sockets are returned in nonblocking mode. 18 | pub fn open_sockets(ip_addr: IpAddr, port: u16, source_peers: usize) -> Result>, ConnectError> { 19 | let listener = bind(ip_addr, port)?; 20 | await_sockets(listener, source_peers) 21 | } 22 | 23 | /// Wraps `TcpListener::bind` to return `ConnectError`s. 24 | pub fn bind(ip_addr: IpAddr, port: u16) -> Result { 25 | let socket_addr = (ip_addr, port).to_socket_addrs()? 26 | .next().ok_or(ConnectError::Other("Invalid listening address".to_string()))?; 27 | 28 | match TcpListener::bind(socket_addr) { 29 | Err(err) => Err(ConnectError::Other(err.to_string())), 30 | Ok(listener) => Ok(listener) 31 | } 32 | } 33 | 34 | /// Listens on the provided socket until `source_peers` connections 35 | /// from the computations we're examining have been established (one 36 | /// socket for every worker on the examined computation). 37 | /// 38 | /// The sockets are wrapped in `Some(_)` because the result is 39 | /// commonly used as a an argument to `make_readers` in this module. 40 | /// 41 | /// The sockets are returned in nonblocking mode. 42 | pub fn await_sockets(listener: TcpListener, source_peers: usize) -> Result>, ConnectError> { 43 | Ok((0..source_peers).map(|_| { 44 | let socket = listener.incoming().next().expect("Socket unexpectedly unavailable"); 45 | if let Ok(ref s) = &socket { 46 | s.set_nonblocking(true)?; 47 | } 48 | socket.map(Some) 49 | }).collect::, _>>()?) 50 | } 51 | 52 | /// Types of Read created by `make_replayers` 53 | pub enum TcpStreamOrFile { 54 | /// a TCP-backed online reader 55 | Tcp(TcpStream), 56 | /// a file-backed offline reader 57 | File(File), 58 | } 59 | 60 | impl std::io::Read for TcpStreamOrFile { 61 | fn read(&mut self, buf: &mut [u8]) -> std::io::Result { 62 | match self { 63 | TcpStreamOrFile::Tcp(x) => x.read(buf), 64 | TcpStreamOrFile::File(x) => x.read(buf), 65 | } 66 | } 67 | } 68 | 69 | /// Source of binary data for `make_replayers`. 70 | #[derive(Clone)] 71 | pub enum ReplaySource { 72 | Tcp(Arc>>>), 73 | Files(Arc>>>), 74 | } 75 | 76 | /// Construct EventReaders that read data from sockets or file 77 | /// and can stream it into timely dataflow. 78 | pub fn make_readers( 79 | source: ReplaySource, 80 | worker_index: usize, 81 | worker_peers: usize, 82 | ) -> Result>, ConnectError> { 83 | 84 | match source { 85 | ReplaySource::Tcp(sockets) => 86 | Ok(sockets.lock().unwrap() 87 | .iter_mut().enumerate() 88 | .filter(|(i, _)| *i % worker_peers == worker_index) 89 | .map(|(_, s)| s.take().expect("socket missing, check the docs for make_replayers")) 90 | .map(|r| EventReader::::new(TcpStreamOrFile::Tcp(r))) 91 | .collect::>()), 92 | ReplaySource::Files(files) => { 93 | let open_files = files.lock().unwrap() 94 | .iter_mut().enumerate() 95 | .filter(|(i, _)| i % worker_peers == worker_index) 96 | .map(|(_, s)| s.take().expect("file name missing, check the docs for make_replayers")) 97 | .map(|p| File::open(&p)) 98 | .collect::, std::io::Error>>()?; 99 | Ok(open_files.into_iter() 100 | .map(|f| EventReader::new(TcpStreamOrFile::File(f))) 101 | .collect::>()) 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /connect/src/receive/mod.rs: -------------------------------------------------------------------------------- 1 | mod connect; 2 | pub use connect::*; 3 | 4 | mod replaywithshutdown; 5 | pub use replaywithshutdown::*; 6 | -------------------------------------------------------------------------------- /connect/src/receive/replaywithshutdown.rs: -------------------------------------------------------------------------------- 1 | // This code adapted from https://github.com/TimelyDataflow/timely-dataflow/blob/master/timely/src/dataflow/operators/capture/replay.rs 2 | // 3 | // Timely Dataflow carries the following license: 4 | // 5 | // The MIT License (MIT) 6 | // 7 | // Copyright (c) 2014 Frank McSherry 8 | // 9 | // Permission is hereby granted, free of charge, to any person obtaining a copy 10 | // of this software and associated documentation files (the "Software"), to deal 11 | // in the Software without restriction, including without limitation the rights 12 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 13 | // copies of the Software, and to permit persons to whom the Software is 14 | // furnished to do so, subject to the following conditions: 15 | // 16 | // The above copyright notice and this permission notice shall be included in all 17 | // copies or substantial portions of the Software. 18 | // 19 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 20 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 21 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 22 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 23 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 24 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 25 | // SOFTWARE. 26 | 27 | use std::sync::{Arc, atomic::AtomicBool, atomic::Ordering}; 28 | 29 | use timely::{Data, dataflow::{Scope, Stream}, progress::Timestamp}; 30 | use timely::dataflow::channels::pushers::{Counter as PushCounter, buffer::Buffer as PushBuffer}; 31 | use timely::dataflow::operators::generic::builder_raw::OperatorBuilder; 32 | use timely::progress::frontier::MutableAntichain; 33 | 34 | use timely::dataflow::operators::capture::event::{Event, EventIterator}; 35 | 36 | /// Replay a capture stream into a scope with the same timestamp. 37 | pub trait ReplayWithShutdown { 38 | /// Replays `self` into the provided scope, as a `Stream`. 39 | fn replay_with_shutdown_into>(self, scope: &mut S, is_running: Arc) -> Stream; 40 | } 41 | 42 | impl ReplayWithShutdown for I 43 | where I : IntoIterator, 44 | ::Item: EventIterator+'static { 45 | fn replay_with_shutdown_into>(self, scope: &mut S, is_running: Arc) -> Stream { 46 | 47 | let mut builder = OperatorBuilder::new("Replay".to_owned(), scope.clone()); 48 | 49 | let address = builder.operator_info().address; 50 | let activator = scope.activator_for(&address[..]); 51 | 52 | let (targets, stream) = builder.new_output(); 53 | 54 | let mut output = PushBuffer::new(PushCounter::new(targets)); 55 | let mut event_streams = self.into_iter().collect::>(); 56 | let mut started = false; 57 | 58 | let mut antichain = MutableAntichain::new(); 59 | 60 | builder.build( 61 | move |progress| { 62 | 63 | if !started { 64 | // The first thing we do is modify our capabilities to match the number of streams we manage. 65 | // This should be a simple change of `self.event_streams.len() - 1`. We only do this once, as 66 | // our very first action. 67 | progress.internals[0].update(Default::default(), (event_streams.len() as i64) - 1); 68 | antichain.update_iter(Some((Default::default(), (event_streams.len() as i64) - 1)).into_iter()); 69 | started = true; 70 | } 71 | 72 | if is_running.load(Ordering::Acquire) { 73 | 74 | for event_stream in event_streams.iter_mut() { 75 | while let Some(event) = event_stream.next() { 76 | match *event { 77 | Event::Progress(ref vec) => { 78 | antichain.update_iter(vec.iter().cloned()); 79 | progress.internals[0].extend(vec.iter().cloned()); 80 | }, 81 | Event::Messages(ref time, ref data) => { 82 | output.session(time).give_iterator(data.iter().cloned()); 83 | } 84 | } 85 | } 86 | } 87 | 88 | // Always reschedule `replay`. 89 | activator.activate(); 90 | 91 | output.cease(); 92 | output.inner().produced().borrow_mut().drain_into(&mut progress.produceds[0]); 93 | 94 | } else { 95 | 96 | while !antichain.is_empty() { 97 | let elements = antichain.frontier().iter().map(|t| (t.clone(), -1)).collect::>(); 98 | for (t, c) in elements.iter() { 99 | progress.internals[0].update(t.clone(), *c); 100 | } 101 | antichain.update_iter(elements); 102 | } 103 | 104 | } 105 | 106 | false 107 | } 108 | ); 109 | 110 | stream 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /examples/pagerank.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/TimelyDataflow/diagnostics/0fa322c98f6982ac6a1a782ad5ac6b5b6a6b8cb5/examples/pagerank.png -------------------------------------------------------------------------------- /tdiag/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | /Cargo.lock 3 | /graph.html 4 | -------------------------------------------------------------------------------- /tdiag/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "tdiag" 3 | version = "0.2.1-pre" 4 | authors = ["Andrea Lattuada "] 5 | homepage = "https://github.com/TimelyDataflow/diagnostics" 6 | repository = "https://github.com/TimelyDataflow/diagnostics.git" 7 | description = "Tools to inspect a timely-dataflow computation" 8 | license = "MIT" 9 | 10 | edition = "2018" 11 | 12 | [dependencies] 13 | timely = "^0.11" 14 | differential-dataflow = "^0.11" 15 | clap = "^2.33" 16 | # tdiag-connect = "^0.2" 17 | tdiag-connect = { path = "../connect" } 18 | -------------------------------------------------------------------------------- /tdiag/release.toml: -------------------------------------------------------------------------------- 1 | upload-doc = false 2 | tag-prefix = "tdiag-" 3 | pre-release-commit-message = "[tdiag] Release version {{version}}" 4 | pro-release-commit-message = "[tdiag] Development version {{next_version}}" 5 | tag-message = "tdiag version {{version}}" 6 | dev-version-ext = "pre" 7 | -------------------------------------------------------------------------------- /tdiag/src/commands/arrangements.rs: -------------------------------------------------------------------------------- 1 | //! "arrangements" subcommand: cli tool to extract logical arrangement 2 | //! sizes over time. 3 | 4 | use std::sync::{Arc, Mutex}; 5 | use std::time::Duration; 6 | use std::convert::TryFrom; 7 | 8 | use crate::DiagError; 9 | 10 | use timely::dataflow::operators::{Filter, Map}; 11 | use timely::logging::{TimelyEvent, WorkerIdentifier}; 12 | use TimelyEvent::Operates; 13 | 14 | use differential_dataflow::collection::AsCollection; 15 | use differential_dataflow::logging::DifferentialEvent; 16 | use differential_dataflow::operators::{Count, Join}; 17 | use DifferentialEvent::{Batch, Merge, MergeShortfall, TraceShare}; 18 | 19 | use tdiag_connect::receive::ReplayWithShutdown; 20 | 21 | /// Prints the number of tuples maintained in each arrangement. 22 | /// 23 | /// 1. Listens to incoming connections from a differential-dataflow 24 | /// program with timely and differential logging enabled; 25 | /// 2. runs a differential-dataflow program to track batching and 26 | /// compaction events and derive number of tuples for each trace; 27 | /// 3. prints the current size alongside arrangement names; 28 | pub fn listen( 29 | timely_configuration: timely::Configuration, 30 | timely_sockets: Vec>, 31 | differential_sockets: Vec>, 32 | output_interval_ms: u64, 33 | ) -> Result<(), crate::DiagError> { 34 | let timely_sockets = Arc::new(Mutex::new(timely_sockets)); 35 | let differential_sockets = Arc::new(Mutex::new(differential_sockets)); 36 | 37 | let is_running = Arc::new(std::sync::atomic::AtomicBool::new(true)); 38 | let is_running_w = is_running.clone(); 39 | 40 | timely::execute(timely_configuration, move |worker| { 41 | let timely_sockets = timely_sockets.clone(); 42 | let differential_sockets = differential_sockets.clone(); 43 | 44 | let timely_replayer = tdiag_connect::receive::make_readers::< 45 | Duration, 46 | (Duration, WorkerIdentifier, TimelyEvent), 47 | >( 48 | tdiag_connect::receive::ReplaySource::Tcp(timely_sockets), 49 | worker.index(), 50 | worker.peers(), 51 | ) 52 | .expect("failed to open timely tcp readers"); 53 | 54 | let differential_replayer = tdiag_connect::receive::make_readers::< 55 | Duration, 56 | (Duration, WorkerIdentifier, DifferentialEvent), 57 | >( 58 | tdiag_connect::receive::ReplaySource::Tcp(differential_sockets), 59 | worker.index(), 60 | worker.peers(), 61 | ) 62 | .expect("failed to open differential tcp readers"); 63 | 64 | worker.dataflow::(|scope| { 65 | let operates = timely_replayer 66 | .replay_with_shutdown_into(scope, is_running_w.clone()) 67 | .flat_map(|(t, worker, x)| { 68 | if let Operates(event) = x { 69 | Some(( 70 | ( 71 | (worker, event.id), 72 | format!("{} ({:?})", event.name, event.addr), 73 | ), 74 | t, 75 | 1 as isize, 76 | )) 77 | } else { 78 | None 79 | } 80 | }) 81 | .as_collection(); 82 | 83 | let events = 84 | differential_replayer.replay_with_shutdown_into(scope, is_running_w.clone()); 85 | 86 | // Print output header. 87 | println!("ms\tWorker\tOp. Id\tName\t# of tuples"); 88 | 89 | // Track sizes. 90 | events 91 | .flat_map(|(t, worker, x)| match x { 92 | Batch(x) => Some(((worker, x.operator), t, x.length as isize)), 93 | Merge(x) => match x.complete { 94 | None => None, 95 | Some(complete_size) => { 96 | let size_diff = 97 | (complete_size as isize) - (x.length1 + x.length2) as isize; 98 | 99 | Some(((worker, x.operator), t, size_diff as isize)) 100 | } 101 | }, 102 | MergeShortfall(x) => { 103 | eprintln!("MergeShortfall {:?}", x); 104 | None 105 | }, 106 | DifferentialEvent::Drop(x) => Some(((worker, x.operator), t, -(x.length as isize))), 107 | TraceShare(_x) => None, 108 | }) 109 | .as_collection() 110 | .delay(move |t| { 111 | let timestamp: u64 = u64::try_from(t.as_millis()) 112 | .expect("Why are the timestamps larger than humans are old?"); 113 | 114 | let window_idx = (timestamp / output_interval_ms) + 1; 115 | 116 | Duration::from_millis(window_idx * output_interval_ms) 117 | }) 118 | .count() 119 | .inner 120 | // We do not bother with retractions here, because the 121 | // user is only interested in the current count. 122 | .filter(|(_, _, count)| count >= &0) 123 | .as_collection() 124 | .join(&operates) 125 | .inspect(|(((worker, operator), (count, name)), t, _diff)| { 126 | println!("{}\t{}\t{}\t{}\t{}", t.as_millis(), worker, operator, name, count); 127 | }); 128 | }) 129 | }) 130 | .map_err(|x| DiagError(format!("error in the timely computation: {}", x)))?; 131 | 132 | Ok(()) 133 | } 134 | -------------------------------------------------------------------------------- /tdiag/src/commands/graph.rs: -------------------------------------------------------------------------------- 1 | //! "graph" subcommand: browser-based tool to display the dataflow graph. 2 | 3 | use std::sync::{Arc, Mutex}; 4 | 5 | use crate::{DiagError, LoggingTuple}; 6 | 7 | use timely::dataflow::operators::{Filter, capture::{Capture, extract::Extract}}; 8 | use timely::dataflow::operators::map::Map; 9 | 10 | use differential_dataflow::collection::AsCollection; 11 | use differential_dataflow::operators::{Join, reduce::Threshold, Consolidate}; 12 | 13 | use timely::logging::TimelyEvent::{Operates, Channels}; 14 | 15 | use tdiag_connect::receive::ReplayWithShutdown; 16 | 17 | static GRAPH_HTML: &str = include_str!("graph/dataflow-graph.html"); 18 | 19 | /// Creates an html file that renders a timely dataflow graph. 20 | /// 21 | /// 1. Listens to incoming connection from a timely-dataflow program with 22 | /// logging enabled; 23 | /// 2. runs a differential-dataflow program to reconstruct the source 24 | /// computation's dataflow graph; 25 | /// 3. writes an html file that renders the dataflow graph. 26 | /// 27 | /// This module includes `graph/dataflow-graph.html` as a static resource. 28 | pub fn listen_and_render( 29 | timely_configuration: timely::Configuration, 30 | sockets: Vec>, 31 | output_path: &std::path::Path) -> Result<(), crate::DiagError> { 32 | 33 | let sockets = Arc::new(Mutex::new(sockets)); 34 | 35 | let (operators_send, operators_recv) = ::std::sync::mpsc::channel(); 36 | let operators_send = Arc::new(Mutex::new(operators_send)); 37 | 38 | let (channels_send, channels_recv) = ::std::sync::mpsc::channel(); 39 | let channels_send = Arc::new(Mutex::new(channels_send)); 40 | 41 | let is_running = Arc::new(std::sync::atomic::AtomicBool::new(true)); 42 | let is_running_w = is_running.clone(); 43 | 44 | let worker_handles = timely::execute(timely_configuration, move |worker| { 45 | let operators_send: std::sync::mpsc::Sender<_> = operators_send.lock().expect("cannot lock operators_send").clone(); 46 | let channels_send: std::sync::mpsc::Sender<_> = channels_send.lock().expect("cannot lock channels_send").clone(); 47 | 48 | let sockets = sockets.clone(); 49 | 50 | // create replayer from disjoint partition of source worker identifiers. 51 | let replayer = tdiag_connect::receive::make_readers::( 52 | tdiag_connect::receive::ReplaySource::Tcp(sockets), worker.index(), worker.peers()) 53 | .expect("failed to open tcp readers"); 54 | 55 | worker.dataflow(|scope| { 56 | let stream = replayer.replay_with_shutdown_into(scope, is_running_w.clone()) 57 | .filter(|(_, worker, _)| *worker == 0); 58 | 59 | let operates = stream 60 | .flat_map(|(t, _, x)| if let Operates(event) = x { Some((event, t, 1 as isize)) } else { None }) 61 | .as_collection(); 62 | 63 | let channels = stream 64 | .flat_map(|(t, _, x)| if let Channels(event) = x { Some((event, t, 1 as isize)) } else { None }) 65 | .as_collection(); 66 | 67 | // == Re-construct the dataflow graph (re-wire channels crossing a scope boundary) == 68 | // 69 | // A timely dataflow graph has a hierarchical structure: a "scope" looks like an 70 | // operator to the outside but can contain a subgraph of operators (and other scopes) 71 | // 72 | // We flatten this hierarchy to display it as a simple directed graph, but preserve the 73 | // information on scope boundaries so that they can be drawn as graph cuts. 74 | 75 | let operates = operates.map(|event| (event.addr, event.name)); 76 | 77 | // Addresses of potential scopes (excluding leaf operators) 78 | let scopes = operates.map(|(mut addr, _)| { 79 | addr.pop(); 80 | addr 81 | }).distinct(); 82 | 83 | // Exclusively leaf operators 84 | let operates_without_subg = operates.antijoin(&scopes); 85 | 86 | // Retain only subscopes that correspond to scopes observed in the logs (remove empty [] addrs) 87 | let subgraphs = operates.map(|(addr, _)| (addr, ())).semijoin(&scopes).map(|(addr, ())| addr); 88 | 89 | let channels = channels.map(|event| (event.id, (event.scope_addr, event.source, event.target))); 90 | 91 | // Output leaf operators 92 | { 93 | operates_without_subg 94 | .consolidate() 95 | .inner 96 | .map(move |((addr, name), _, _)| (addr, name)) 97 | .capture_into(operators_send); 98 | } 99 | 100 | // Output channels 101 | { 102 | // Channels that enter a subscope (as seen from outside the subscope) 103 | // Their source is the operator representing the subscope 104 | let subg_channels_outside_ingress = channels 105 | .map(|(id, (scope_addr, from, to))| { 106 | let mut subscope_addr = scope_addr.clone(); 107 | subscope_addr.push(to.0); 108 | (subscope_addr, (id, (scope_addr, from), to.1)) 109 | }) 110 | .semijoin(&subgraphs); 111 | 112 | // Channels that leave a subscope (as seen from outside the subscope) 113 | // Their destination is the operator representing the subscope 114 | let subg_channels_outside_egress = channels 115 | .map(|(id, (scope_addr, from, to))| { 116 | let mut subscope_addr = scope_addr.clone(); 117 | subscope_addr.push(from.0); 118 | (subscope_addr, (id, from.1, (scope_addr, to))) 119 | }) 120 | .semijoin(&subgraphs); 121 | 122 | // Join the external and internal representation of channels that enter a subscope 123 | // 124 | // subscope [0, 1, 4] 125 | // +-----------------------+ 126 | // ([0, 1], 3) | ([0, 1, 4], 1) | 127 | // o------------->|------------->o | 128 | // external | internal | 129 | // +-----------------------+ 130 | // 131 | // The external channel has addr [0, 1], source 3, destination 4 ([0, 1, 4] is the subscope). 132 | // The internal channel has addr [0, 1, 4], source 0 (special!), destination 1. 133 | let subg_ingress = subg_channels_outside_ingress 134 | .map(|(subscope_addr, (id, orig, subscope_port))| ((subscope_addr, (0, subscope_port)), (id, orig))) 135 | .join_map( 136 | &channels.map(|(id, (scope_addr, from, to))| ((scope_addr, from), (id, to))), 137 | |(scope_addr, _from), (id1, (orig_addr, orig_from)), (id2, to)| { 138 | let mut orig_addr = orig_addr.clone(); 139 | orig_addr.push(orig_from.0); 140 | let mut to_addr = scope_addr.clone(); 141 | to_addr.push(to.0); 142 | (vec![*id1, *id2], true, orig_addr, to_addr, orig_from.1, to.1) 143 | }); 144 | 145 | // Join the external and internal representation of channels that leave a subscope 146 | // 147 | // The structure depicted above is inverted for channels leaving a subscope 148 | let subg_egress = subg_channels_outside_egress 149 | .map(|(subscope_addr, (id, subscope_port, dest))| ((subscope_addr, (0, subscope_port)), (id, dest))) 150 | .join_map( 151 | &channels.map(|(id, (scope_addr, from, to))| ((scope_addr, to), (id, from))), 152 | |(scope_addr, to), (id2, (dest_addr, dest_to)), (id1, from)| { 153 | let mut from_addr = scope_addr.clone(); 154 | from_addr.push(from.0); 155 | let mut dest_addr = dest_addr.clone(); 156 | dest_addr.push(dest_to.0); 157 | (vec![*id1, *id2], true, from_addr, dest_addr, to.1, dest_to.1) 158 | }); 159 | 160 | // Select all other channels (those that don't enter/leave a subscope) 161 | let non_subg = channels 162 | .map(|(id, (scope_addr, from, to))| { 163 | let mut subscope_addr = scope_addr.clone(); 164 | subscope_addr.push(from.0); 165 | (subscope_addr, (id, scope_addr, from, to)) 166 | }) 167 | .antijoin(&subgraphs) 168 | .map(|(_, (id, scope_addr, from, to))| { 169 | let mut subscope_addr = scope_addr.clone(); 170 | subscope_addr.push(to.0); 171 | (subscope_addr, (id, scope_addr, from, to)) 172 | }) 173 | .antijoin(&subgraphs) 174 | .map(|(_, (id, scope_addr, from, to))| { 175 | let mut from_addr = scope_addr.clone(); 176 | from_addr.push(from.0); 177 | let mut to_addr = scope_addr.clone(); 178 | to_addr.push(to.0); 179 | (vec![id], false, from_addr, to_addr, from.1, to.1) 180 | }); 181 | 182 | subg_ingress 183 | .concat(&subg_egress) 184 | .concat(&non_subg) 185 | .consolidate() 186 | .inner 187 | .map(|(x, _, _)| x) 188 | .capture_into(channels_send); 189 | 190 | } 191 | }) 192 | }).map_err(|x| DiagError(format!("error in the timely computation: {}", x)))?; 193 | 194 | { 195 | use std::io; 196 | use std::io::prelude::*; 197 | 198 | let mut stdin = io::stdin(); 199 | let mut stdout = io::stdout(); 200 | 201 | write!(stdout, "Press enter to generate graph (this will crash the source computation if it hasn't terminated).") 202 | .expect("failed to write to stdout"); 203 | stdout.flush().unwrap(); 204 | 205 | // Read a single byte and discard 206 | let _ = stdin.read(&mut [0u8]).expect("failed to read from stdin"); 207 | } 208 | 209 | is_running.store(false, std::sync::atomic::Ordering::Release); 210 | 211 | worker_handles.join().into_iter().collect::, _>>().expect("Timely error"); 212 | 213 | let mut file = std::fs::File::create(output_path).map_err(|e| DiagError(format!("io error: {}", e)))?; 214 | 215 | use std::io::Write; 216 | 217 | fn expect_write(e: Result<(), std::io::Error>) { 218 | e.expect("write failed"); 219 | } 220 | 221 | expect_write(writeln!(file, "")); 222 | expect_write(writeln!(file, "{}", GRAPH_HTML)); 223 | expect_write(writeln!(file, "")); 252 | 253 | println!("Graph generated in file://{}", std::fs::canonicalize(output_path).expect("invalid path").to_string_lossy()); 254 | 255 | Ok(()) 256 | } 257 | -------------------------------------------------------------------------------- /tdiag/src/commands/graph/dataflow-graph.html: -------------------------------------------------------------------------------- 1 | 28 | 29 | 30 | 31 | 98 | 99 | -------------------------------------------------------------------------------- /tdiag/src/commands/mod.rs: -------------------------------------------------------------------------------- 1 | //! Subcommand modules. 2 | //! 3 | //! Each of the program's subcommand logic is in a separate module here. 4 | //! 5 | //! Subfolders in the source tree contain resource files. 6 | 7 | pub mod graph; 8 | pub mod profile; 9 | pub mod arrangements; 10 | -------------------------------------------------------------------------------- /tdiag/src/commands/profile.rs: -------------------------------------------------------------------------------- 1 | //! "profile" subcommand: reports aggregate runtime for each 2 | //! scope/operator. 3 | 4 | use std::sync::{Arc, Mutex}; 5 | 6 | use crate::{DiagError, LoggingTuple}; 7 | 8 | use timely::dataflow::operators::{Map, Filter, generic::Operator}; 9 | 10 | use differential_dataflow::trace::TraceReader; 11 | use differential_dataflow::collection::AsCollection; 12 | use differential_dataflow::operators::{Join, reduce::Threshold, Consolidate, arrange::{Arrange, Arranged}}; 13 | 14 | use timely::logging::TimelyEvent::{Operates, Schedule}; 15 | 16 | use tdiag_connect::receive::ReplayWithShutdown; 17 | 18 | /// Prints aggregate time spent in each scope/operator. 19 | /// 20 | /// 1. Listens to incoming connections from a timely-dataflow program 21 | /// with logging enabled; 22 | /// 2. runs a differential-dataflow program to track scheduling events 23 | /// and derive runtime for each operator; 24 | /// 3. prints the resulting measurements alongside operator names and 25 | /// scope names; 26 | pub fn listen_and_profile( 27 | timely_configuration: timely::Configuration, 28 | sockets: Vec>) -> Result<(), crate::DiagError> { 29 | 30 | let sockets = Arc::new(Mutex::new(sockets)); 31 | 32 | let (output_send, output_recv) = ::std::sync::mpsc::channel(); 33 | let output_send = Arc::new(Mutex::new(output_send)); 34 | 35 | let is_running = Arc::new(std::sync::atomic::AtomicBool::new(true)); 36 | let is_running_w = is_running.clone(); 37 | 38 | let worker_handles = timely::execute(timely_configuration, move |worker| { 39 | let output_send: std::sync::mpsc::Sender<_> = output_send.lock().expect("cannot lock output_send").clone(); 40 | 41 | let sockets = sockets.clone(); 42 | 43 | // create replayer from disjoint partition of source worker identifiers. 44 | let replayer = tdiag_connect::receive::make_readers::( 45 | tdiag_connect::receive::ReplaySource::Tcp(sockets), worker.index(), worker.peers()) 46 | .expect("failed to open tcp readers"); 47 | 48 | let profile_trace = worker.dataflow(|scope| { 49 | let stream = replayer.replay_with_shutdown_into(scope, is_running_w.clone()); 50 | 51 | let operates = stream 52 | .filter(|(_, w, _)| *w== 0) 53 | .flat_map(|(t, _, x)| if let Operates(event) = x { Some((event, t, 1 as isize)) } else { None }) 54 | .as_collection(); 55 | 56 | let schedule = stream 57 | .flat_map(|(t, w, x)| if let Schedule(event) = x { Some((t, w, event)) } else { None }) 58 | .unary(timely::dataflow::channels::pact::Pipeline, "Schedules", |_,_| { 59 | let mut map = std::collections::HashMap::new(); 60 | let mut vec = Vec::new(); 61 | move |input, output| { 62 | input.for_each(|time, data| { 63 | data.swap(&mut vec); 64 | let mut session = output.session(&time); 65 | for (ts, worker, event) in vec.drain(..) { 66 | let key = (worker, event.id); 67 | match event.start_stop { 68 | timely::logging::StartStop::Start => { 69 | assert!(!map.contains_key(&key)); 70 | map.insert(key, ts); 71 | }, 72 | timely::logging::StartStop::Stop => { 73 | assert!(map.contains_key(&key)); 74 | let end = map.remove(&key).unwrap(); 75 | let ts_clip = std::time::Duration::from_secs(ts.as_secs() + 1); 76 | let elapsed = ts - end; 77 | let elapsed_ns = (elapsed.as_secs() as isize) * 1_000_000_000 + (elapsed.subsec_nanos() as isize); 78 | session.give((key.1, ts_clip, elapsed_ns)); 79 | } 80 | } 81 | } 82 | }); 83 | } 84 | }).as_collection().consolidate(); // (operator_id) 85 | 86 | // FIXME 87 | // == Re-construct the dataflow graph (re-wire channels crossing a scope boundary) == 88 | // 89 | // A timely dataflow graph has a hierarchical structure: a "scope" looks like an 90 | // operator to the outside but can contain a subgraph of operators (and other scopes) 91 | // 92 | // We flatten this hierarchy to display it as a simple directed graph, but preserve the 93 | // information on scope boundaries so that they can be drawn as graph cuts. 94 | 95 | let operates = operates.map(|event| (event.addr, (event.id, event.name))); 96 | 97 | // Addresses of potential scopes (excluding leaf operators) 98 | let scopes = operates.map(|(mut addr, _)| { 99 | addr.pop(); 100 | addr 101 | }).distinct(); 102 | 103 | // Exclusively leaf operators 104 | let operates_without_subg = operates.antijoin(&scopes).map(|(addr, (id, name))| (id, (addr, name, false))); 105 | let subg = operates.semijoin(&scopes).map(|(addr, (id, name))| (id, (addr, name, true))); 106 | 107 | let all_operators = operates_without_subg.concat(&subg).distinct(); 108 | 109 | use differential_dataflow::trace::implementations::ord::OrdKeySpine; 110 | let Arranged { trace: profile_trace, .. } = all_operators.semijoin(&schedule) 111 | .map(|(id, (addr, name, is_scope))| (id, addr, name, is_scope)) 112 | .consolidate() 113 | .arrange::>(); 114 | 115 | profile_trace 116 | }); 117 | 118 | while worker.step() { } 119 | 120 | let mut profile_trace = profile_trace; 121 | 122 | profile_trace.distinguish_since(&[]); 123 | 124 | let (mut cursor, storage) = profile_trace.cursor(); 125 | 126 | use differential_dataflow::trace::cursor::Cursor; 127 | while cursor.key_valid(&storage) { 128 | let key = cursor.key(&storage); 129 | if cursor.val_valid(&storage) { 130 | let mut ns = 0; 131 | cursor.map_times(&storage, |_, r| ns += r); 132 | output_send.send((key.clone(), ns)).expect("failed to send output to mpsc channel"); 133 | } 134 | cursor.step_key(&storage); 135 | } 136 | 137 | }).map_err(|x| DiagError(format!("error in the timely computation: {}", x)))?; 138 | 139 | { 140 | use std::io; 141 | use std::io::prelude::*; 142 | 143 | let mut stdin = io::stdin(); 144 | let mut stdout = io::stdout(); 145 | 146 | write!(stdout, "Press enter to stop collecting profile data (this will crash the source computation if it hasn't terminated).") 147 | .expect("failed to write to stdout"); 148 | stdout.flush().unwrap(); 149 | 150 | // Read a single byte and discard 151 | let _ = stdin.read(&mut [0u8]).expect("failed to read from stdin"); 152 | } 153 | 154 | is_running.store(false, std::sync::atomic::Ordering::Release); 155 | 156 | worker_handles.join().into_iter().collect::, _>>().expect("Timely error"); 157 | 158 | let mut data = output_recv.into_iter().collect::>(); 159 | data.sort_unstable_by_key(|&(_, ns)| std::cmp::Reverse(ns)); 160 | for ((id, addr, name, is_scope), ns) in data.into_iter() { 161 | println!("{}\t{}\t(id={}, addr={:?}):\t{:e} s", 162 | if is_scope { "[scope]" } else { "" }, 163 | name, 164 | id, 165 | addr, 166 | (ns as f64) / 1_000_000_000f64); 167 | } 168 | 169 | Ok(()) 170 | } 171 | -------------------------------------------------------------------------------- /tdiag/src/lib.rs: -------------------------------------------------------------------------------- 1 | //! Tools to inspect a timely-dataflow computation. 2 | 3 | #![deny(missing_docs)] 4 | 5 | pub mod commands; 6 | 7 | /// An error generated by one of the commands of this tool. 8 | pub struct DiagError(pub String); 9 | 10 | impl From for DiagError { 11 | fn from(error: std::io::Error) -> Self { 12 | DiagError(format!("io error: {}", error.to_string())) 13 | } 14 | } 15 | 16 | impl From for DiagError { 17 | fn from(error: tdiag_connect::ConnectError) -> Self { 18 | match error { 19 | tdiag_connect::ConnectError::IoError(e) => DiagError(format!("io error: {}", e)), 20 | tdiag_connect::ConnectError::Other(e) => DiagError(e), 21 | } 22 | } 23 | } 24 | 25 | type LoggingTuple = (std::time::Duration, timely::logging::WorkerIdentifier, timely::logging::TimelyEvent); 26 | -------------------------------------------------------------------------------- /tdiag/src/main.rs: -------------------------------------------------------------------------------- 1 | //! Command-line tools (with browser-based visualization) 2 | //! to inspect a timely-dataflow computation. 3 | //! 4 | //! See the README and --help for usage information. 5 | 6 | use tdiag::*; 7 | 8 | fn run() -> Result<(), DiagError> { 9 | let args = clap::App::new("tdiag") 10 | .about( 11 | "Diagostic tools for timely-dataflow programs. 12 | Run the timely program to inspect with `env TIMELY_WORKER_LOG_ADDR=127.0.0.1:51317 cargo run ...`. 13 | You can customize the interface and port for the receiver (this program) with --interface and --port. 14 | " 15 | ) 16 | .arg(clap::Arg::with_name("interface") 17 | .short("i") 18 | .long("interface") 19 | .value_name("INTERFACE") 20 | .help("Interface (ip address) to listen on; defaults to 127.0.0.1") 21 | .default_value("127.0.0.1") 22 | .takes_value(true)) 23 | .arg(clap::Arg::with_name("port") 24 | .short("p") 25 | .long("port") 26 | .value_name("PORT") 27 | .help("Port to listen on; defaults to 51317") 28 | .default_value("51317") 29 | .required(true)) 30 | .arg(clap::Arg::with_name("source_peers") 31 | .short("s") 32 | .long("source-peers") 33 | .value_name("PEERS") 34 | .help("Number of workers in the source computation") 35 | .required(true)) 36 | .arg(clap::Arg::with_name("diag_workers") 37 | .short("w") 38 | .long("diag-workers") 39 | .value_name("WORKERS") 40 | .help("Number of worker threads for the diagnostic tool") 41 | .default_value("1")) 42 | .subcommand(clap::SubCommand::with_name("graph") 43 | .about("Render a computation's dataflow graph") 44 | .arg(clap::Arg::with_name("output_path") 45 | .short("o") 46 | .long("out") 47 | .value_name("PATH") 48 | .help("The output path for the generated html file (don't forget the .html extension)") 49 | .required(true)) 50 | ) 51 | .subcommand( 52 | clap::SubCommand::with_name("profile") 53 | .about("Print total time spent running each operator") 54 | ) 55 | .subcommand( 56 | clap::SubCommand::with_name("differential") 57 | .about("Tools for profiling Timely computations that make use of differential dataflow.") 58 | .arg(clap::Arg::with_name("port") 59 | .short("p") 60 | .long("port") 61 | .value_name("PORT") 62 | .help("Port to listen on for Differential log streams; defaults to 51318") 63 | .default_value("51318") 64 | .required(true)) 65 | .subcommand( 66 | clap::SubCommand::with_name("arrangements") 67 | .about("Track the logical size of arrangements over the course of a computation") 68 | .arg(clap::Arg::with_name("output-interval") 69 | .long("output-interval") 70 | .value_name("MS") 71 | .help("Interval (in ms) at which to print arrangement sizes; defaults to 1000ms") 72 | .default_value("1000")) 73 | .after_help(" 74 | Add the following snippet to your Differential computation: 75 | 76 | ``` 77 | if let Ok(addr) = ::std::env::var(\"DIFFERENTIAL_LOG_ADDR\") { 78 | if let Ok(stream) = ::std::net::TcpStream::connect(&addr) { 79 | differential_dataflow::logging::enable(worker, stream); 80 | info!(\"enabled DIFFERENTIAL logging to {}\", addr); 81 | } else { 82 | panic!(\"Could not connect to differential log address: {:?}\", addr); 83 | } 84 | } 85 | ``` 86 | 87 | Then start your computation with the DIFFERENTIAL_LOG_ADDR environment 88 | variable pointing to tdiag's differential port (51318 by default). 89 | ") 90 | ) 91 | ) 92 | .get_matches(); 93 | 94 | match args.subcommand() { 95 | (_, None) => Err(DiagError("Invalid subcommand".to_string()))?, 96 | _ => (), 97 | } 98 | 99 | let ip_addr: std::net::IpAddr = args.value_of("interface").expect("error parsing args") 100 | .parse().map_err(|e| DiagError(format!("Invalid --interface: {}", e)))?; 101 | let port: u16 = args.value_of("port").expect("error parsing args") 102 | .parse().map_err(|e| DiagError(format!("Invalid --port: {}", e)))?; 103 | let source_peers: usize = args.value_of("source_peers").expect("error parsing args") 104 | .parse().map_err(|e| DiagError(format!("Invalid --source-peers: {}", e)))?; 105 | let diag_workers: usize = args.value_of("diag_workers").expect("error parsing args") 106 | .parse().map_err(|e| DiagError(format!("Invalid --diag-workers: {}", e)))?; 107 | 108 | let timely_configuration = match diag_workers { 109 | 1 => timely::Configuration::Thread, 110 | n => timely::Configuration::Process(n), 111 | }; 112 | 113 | match args.subcommand() { 114 | ("graph", Some(graph_args)) => { 115 | let output_path = std::path::Path::new(graph_args.value_of("output_path").expect("error parsing args")); 116 | println!("Listening for {} connections on {}:{}", source_peers, ip_addr, port); 117 | let sockets = tdiag_connect::receive::open_sockets(ip_addr, port, source_peers)?; 118 | println!("Trace sources connected"); 119 | crate::commands::graph::listen_and_render(timely_configuration, sockets, output_path) 120 | } 121 | ("profile", Some(_profile_args)) => { 122 | println!("Listening for {} connections on {}:{}", source_peers, ip_addr, port); 123 | let sockets = tdiag_connect::receive::open_sockets(ip_addr, port, source_peers)?; 124 | println!("Trace sources connected"); 125 | crate::commands::profile::listen_and_profile(timely_configuration, sockets) 126 | } 127 | ("differential", Some(differential_args)) => { 128 | 129 | let differential_port: u16 = differential_args.value_of("port") 130 | .expect("error parsing args") 131 | .parse() 132 | .map_err(|e| DiagError(format!("Invalid --port: {}", e)))?; 133 | 134 | match differential_args.subcommand() { 135 | ("arrangements", Some(args)) => { 136 | // It's crucial that we bind to both listening 137 | // addresses first, before waiting for 138 | // connections. Otherwise we will open up the 139 | // potential for a race condition in the source 140 | // computation. 141 | 142 | println!("Listening for {} Timely connections on {}:{}", source_peers, ip_addr, port); 143 | let timely_listener = tdiag_connect::receive::bind(ip_addr, port)?; 144 | 145 | println!("Listening for {} Differential connections on {}:{}", source_peers, ip_addr, differential_port); 146 | let differential_listener = tdiag_connect::receive::bind(ip_addr, differential_port)?; 147 | 148 | let timely_sockets = tdiag_connect::receive::await_sockets(timely_listener, source_peers)?; 149 | let differential_sockets = tdiag_connect::receive::await_sockets(differential_listener, source_peers)?; 150 | 151 | let output_interval_ms: u64 = args.value_of("output-interval") 152 | .expect("error parsing args") 153 | .parse() 154 | .expect("error parsing args"); 155 | 156 | println!("Will report every {}ms", output_interval_ms); 157 | 158 | println!("Trace sources connected"); 159 | crate::commands::arrangements::listen( 160 | timely_configuration, 161 | timely_sockets, 162 | differential_sockets, 163 | output_interval_ms, 164 | ) 165 | } 166 | _ => panic!("Invalid subcommand for differential diagnostics"), 167 | } 168 | } 169 | _ => panic!("Invalid subcommand"), 170 | } 171 | } 172 | 173 | fn main() { 174 | match run() { 175 | Ok(()) => (), 176 | Err(DiagError(e)) => eprintln!("Error: {}", e), 177 | } 178 | } 179 | --------------------------------------------------------------------------------