├── .gitignore ├── Cargo.toml ├── LICENSE ├── README.md ├── examples ├── basic.rs └── logging-recv2.rs ├── html ├── dashboard.html ├── graph.html └── schedule.html └── src ├── bin ├── dashboard.rs ├── graph.rs └── schedule.rs ├── lib.rs └── server.rs /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | **/*.rs.bk 3 | Cargo.lock 4 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "timely-viz" 3 | version = "0.1.0" 4 | authors = ["Andrea Lattuada "] 5 | 6 | 7 | [dependencies] 8 | abomonation = "0.7" 9 | abomonation_derive = "0.3" 10 | timely_communication = "^0.8" 11 | #timely = "^0.8" 12 | timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } 13 | #differential-dataflow = "^0.8" 14 | differential-dataflow = { git = "https://github.com/TimelyDataflow/differential-dataflow" } 15 | fnv="1.0.2" 16 | futures = "^0.1" 17 | actix = "^0.5" 18 | actix-web = "^0.6" 19 | serde = "^1" 20 | serde_derive = "^1" 21 | serde_json = "^1" 22 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2018 Andrea Lattuada, ETH Zürich 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 | **NOTE: These tools are deprecated in favour of [timelydataflow/diagnostics](https://github.com/timelydataflow/diagnostics).** 2 | 3 | # [→ timelydataflow/diagnostics](https://github.com/timelydataflow/diagnostics) 4 | -------------------------------------------------------------------------------- /examples/basic.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | extern crate timely_viz; 4 | 5 | extern crate abomonation; 6 | 7 | use std::time::Duration; 8 | 9 | use timely::dataflow::operators::{Map, capture::Replay}; 10 | use timely::logging::TimelyEvent::{Operates, Channels}; 11 | 12 | use differential_dataflow::collection::AsCollection; 13 | 14 | fn main() { 15 | let mut args = ::std::env::args(); 16 | args.next().unwrap(); 17 | 18 | // the number of workers in the computation we're examining 19 | let source_peers = args.next().unwrap().parse::().unwrap(); 20 | // one socket per worker in the computation we're examining 21 | let sockets = timely_viz::open_sockets(source_peers); 22 | 23 | timely::execute_from_args(args, move |worker| { 24 | let sockets = sockets.clone(); 25 | 26 | let replayers = timely_viz::make_replayers(sockets, worker.index(), worker.peers()); 27 | 28 | worker.dataflow::(|scope| { 29 | let stream = 30 | replayers 31 | .replay_into(scope); 32 | 33 | let _operates = 34 | stream 35 | .flat_map(|(t,_,x)| { 36 | if let Operates(event) = x { 37 | Some((event, t, 1 as isize)) 38 | } else { 39 | None 40 | } 41 | }) 42 | .as_collection() 43 | // only keep elements that came from worker 0 44 | // (the first element of "addr" is the worker id) 45 | .filter(|x| *x.addr.first().unwrap() == 0) 46 | .inspect(|x| println!("Operates: {:?}", x)); 47 | 48 | let _channels = 49 | stream 50 | .flat_map(|(t,_,x)| { 51 | if let Channels(event) = x { 52 | Some((event, t, 1 as isize)) 53 | } else { 54 | None 55 | } 56 | }) 57 | .as_collection() 58 | .filter(|x| *x.scope_addr.first().unwrap() == 0) 59 | .inspect(|x| println!("Channels: {:?}", x)); 60 | }); 61 | }).unwrap(); 62 | } 63 | -------------------------------------------------------------------------------- /examples/logging-recv2.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | 4 | use std::sync::{Arc, Mutex}; 5 | use std::net::TcpListener; 6 | use std::time::Duration; 7 | 8 | use timely::dataflow::operators::Map; 9 | use timely::logging::TimelyEvent; 10 | use timely::communication::logging::{CommunicationEvent, CommunicationSetup}; 11 | use timely::dataflow::operators::Filter; 12 | use timely::dataflow::operators::capture::{EventReader, Replay}; 13 | 14 | use differential_dataflow::AsCollection; 15 | use differential_dataflow::operators::Consolidate; 16 | 17 | fn main() { 18 | 19 | let mut args = ::std::env::args(); 20 | args.next().unwrap(); 21 | 22 | let work_peers: usize = args.next().expect("Must provide number of source peers").parse().expect("Source peers must be an unsigned integer"); 23 | let comm_peers: usize = args.next().expect("Must provide number of source peers").parse().expect("Comms peers must be an unsigned integer"); 24 | let granularity: u64 = args.next().expect("Must provide report granularity in seconds").parse().expect("Granularity must be an unsigned integer"); 25 | 26 | println!("starting with work peers: {}, comm peers: {}, granularity: {}", work_peers, comm_peers, granularity); 27 | 28 | let t_listener = TcpListener::bind("127.0.0.1:8000").unwrap(); 29 | let d_listener = TcpListener::bind("127.0.0.1:9000").unwrap(); 30 | let t_sockets = 31 | Arc::new(Mutex::new((0..work_peers).map(|_| { 32 | let socket = t_listener.incoming().next().unwrap().unwrap(); 33 | socket.set_nonblocking(true).expect("failed to set nonblocking"); 34 | Some(socket) 35 | }).collect::>())); 36 | let d_sockets = 37 | Arc::new(Mutex::new((0..comm_peers).map(|_| { 38 | let socket = d_listener.incoming().next().unwrap().unwrap(); 39 | socket.set_nonblocking(true).expect("failed to set nonblocking"); 40 | Some(socket) 41 | }).collect::>())); 42 | 43 | timely::execute_from_args(std::env::args(), move |worker| { 44 | 45 | let index = worker.index(); 46 | let peers = worker.peers(); 47 | 48 | let t_streams = 49 | t_sockets 50 | .lock() 51 | .unwrap() 52 | .iter_mut() 53 | .enumerate() 54 | .filter(|(i, _)| *i % peers == index) 55 | .map(move |(_, s)| s.take().unwrap()) 56 | .map(|r| EventReader::::new(r)) 57 | .collect::>(); 58 | 59 | let d_streams = 60 | d_sockets 61 | .lock() 62 | .unwrap() 63 | .iter_mut() 64 | .enumerate() 65 | .filter(|(i, _)| *i % peers == index) 66 | .map(move |(_, s)| s.take().unwrap()) 67 | .map(|r| EventReader::::new(r)) 68 | .collect::>(); 69 | 70 | worker.dataflow::<_,_,_>(|scope| { 71 | 72 | let t_events = t_streams.replay_into(scope); 73 | let d_events = d_streams.replay_into(scope); 74 | 75 | // println!("SIZEOF: {}", ::std::mem::size_of::<(Duration, usize, TimelyEvent)>()); 76 | 77 | t_events 78 | .flat_map(move |(ts, _worker, datum)| { 79 | let ts = Duration::from_secs((ts.as_secs()/granularity + 1) * granularity); 80 | Some((datum, ts, 1)) 81 | }) 82 | .as_collection() 83 | .map(|x| ()) 84 | .consolidate() 85 | .inspect(|x| println!("TIMELY\t{:?}", x)); 86 | 87 | // d_events 88 | // .flat_map(move |(ts, _worker, datum)| { 89 | // let ts = Duration::from_secs((ts.as_secs()/granularity + 1) * granularity); 90 | // Some((datum, ts, 1)) 91 | // }) 92 | // .as_collection() 93 | // .consolidate() 94 | // .inspect(|x| println!("COMMS\t{:?}", x)); 95 | 96 | }); 97 | 98 | }).unwrap(); // asserts error-free execution 99 | } 100 | -------------------------------------------------------------------------------- /html/dashboard.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 |
9 | 10 |
11 |
12 | 13 |
14 |

Description

15 |
    16 |
  • 17 | Operator name: 18 | 19 |
  • 20 |
  • 21 | Elapsed time in ns: 22 | 23 |
  • 24 |
  • 25 | Total count: 26 | 27 |
  • 28 |
      29 |
    30 |
31 |
32 |
33 |
34 |
35 |
36 | 37 | 96 | 97 | 575 | 576 | -------------------------------------------------------------------------------- /html/graph.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 79 | 100 | 101 | 123 | 124 | 125 | -------------------------------------------------------------------------------- /html/schedule.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 |
8 | 9 | 30 | 31 | 32 | 201 | -------------------------------------------------------------------------------- /src/bin/dashboard.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | extern crate actix; 4 | extern crate actix_web; 5 | extern crate futures; 6 | extern crate timely_viz; 7 | 8 | extern crate abomonation; 9 | #[macro_use] extern crate abomonation_derive; 10 | 11 | extern crate serde; 12 | #[macro_use] extern crate serde_derive; 13 | extern crate serde_json; 14 | 15 | use std::sync::{Arc, Mutex}; 16 | use std::time::Duration; 17 | 18 | use timely::dataflow::operators::{Map, capture::Replay}; 19 | use timely::logging::TimelyEvent::{Operates, Schedule, Channels, Messages}; 20 | use timely::dataflow::operators::{Operator, Concat, Filter}; 21 | 22 | use differential_dataflow::AsCollection; 23 | use differential_dataflow::operators::{Count, Consolidate}; 24 | 25 | use actix::*; 26 | use actix_web::*; 27 | use actix_web::{App, HttpRequest, Result, http::Method, fs::NamedFile}; 28 | 29 | use futures::future::Future; 30 | 31 | // ==== Web-server setup ==== 32 | use timely_viz::server::Ws; 33 | 34 | // #[derive(Serialize, Clone, Ord, PartialOrd, Eq, PartialEq)] 35 | // struct Update { 36 | // data: String, // name of data source. 37 | // name: String, 38 | // addr: String, 39 | // parent_addr: String, 40 | // size: isize, 41 | // diff: isize, 42 | // } 43 | 44 | 45 | #[derive(Serialize, Abomonation, Clone, PartialOrd, PartialEq, Eq, Ord, Debug)] 46 | struct Update { 47 | event: Event, 48 | delta: isize, 49 | } 50 | 51 | #[derive(Serialize, Abomonation, Clone, PartialOrd, PartialEq, Eq, Ord, Debug)] 52 | enum Event { 53 | // Largely static 54 | Operate { 55 | id: usize, // unique identifier 56 | addr: Vec, // (x, .., y, z) 57 | name: String, // Helpful name 58 | }, 59 | // Largely static 60 | Channel { 61 | id: usize, // unique identifier 62 | from_addr: Vec, // (x, .., y, z1) 63 | from_port: usize, // port1 64 | to_addr: Vec, // (x, .., y, z2) 65 | to_port: usize, // port2 66 | }, 67 | Schedule { 68 | operate_id: usize, 69 | elapsed_ns: isize, // total elapsed ns 70 | }, 71 | Messages { 72 | channel_id: usize, 73 | total_count: isize, // total typed records 74 | } 75 | } 76 | 77 | 78 | #[derive(Serialize, Clone)] 79 | struct Updates { 80 | updates: Vec, 81 | } 82 | 83 | impl actix::Message for Updates { 84 | type Result = (); 85 | } 86 | 87 | impl Handler for Ws { 88 | type Result = (); 89 | 90 | fn handle(&mut self, msg: Updates, ctx: &mut Self::Context) { 91 | let text = serde_json::to_string(&msg).unwrap(); 92 | // println!("Sending: {}", text); 93 | ctx.text(text); 94 | } 95 | } 96 | // ========================== 97 | 98 | fn main() { 99 | let endpoint: Arc>>> = Arc::new(Mutex::new(Vec::new())); 100 | let endpoint_actix = endpoint.clone(); 101 | 102 | ::std::mem::drop(::std::thread::spawn(move || { 103 | 104 | let mut args = ::std::env::args(); 105 | args.next().unwrap(); 106 | 107 | // the number of workers in the computation we're examining 108 | let source_peers = args.next().expect("Must provide number of source peers").parse::().expect("Source peers must be an unsigned integer"); 109 | let frequency_ns = args.next().expect("Must provide nanoseconds frequency").parse::().expect("Frequency must be an unsigned integer"); 110 | let sockets = timely_viz::open_sockets(source_peers); 111 | 112 | timely::execute_from_args(std::env::args(), move |worker| { 113 | 114 | let endpoint = endpoint.clone(); 115 | 116 | let sockets = sockets.clone(); 117 | 118 | // create replayers from disjoint partition of source worker identifiers 119 | let replayers = timely_viz::make_replayers(sockets, worker.index(), worker.peers()); 120 | 121 | worker.dataflow::(|scope| { 122 | 123 | let replayed = replayers.replay_into(scope); 124 | 125 | let operates = 126 | replayed 127 | .flat_map(move |(mut ts, _setup, datum)| 128 | if let Operates(mut event) = datum { 129 | event.addr.insert(0, _setup); 130 | ts = round_duration_up_to_ns(ts, frequency_ns); 131 | Some((event, ts, 1)) 132 | } 133 | else { None } 134 | ) 135 | .as_collection() 136 | .filter(|x| x.addr[0] == 0) 137 | .map(|x| (x.id, x.addr, x.name)) 138 | .consolidate() 139 | .inner 140 | .map(|((id, addr, name), _time, diff)| { 141 | 142 | Update { 143 | event: Event::Operate { 144 | id, 145 | addr, 146 | name, 147 | }, 148 | delta: diff, 149 | } 150 | }); 151 | 152 | let channels = 153 | replayed 154 | .flat_map(move |(mut ts,_setup,x)| 155 | if let Channels(mut event) = x { 156 | event.scope_addr.insert(0, _setup); 157 | ts = round_duration_up_to_ns(ts, frequency_ns); 158 | Some(((event.id, event.scope_addr, event.source, event.target), ts, 1 as isize)) 159 | } 160 | else { 161 | None 162 | } 163 | ) 164 | .as_collection() 165 | .consolidate() 166 | .inner 167 | .filter(|x| (x.0).1[0] == 0) 168 | .map(|((id, scope_addr, source, target), _time, diff)| { 169 | 170 | let mut from_addr = scope_addr.clone(); 171 | from_addr.push(source.0); 172 | 173 | let mut to_addr = scope_addr.clone(); 174 | to_addr.push(target.0); 175 | 176 | Update { 177 | event: Event::Channel { 178 | id, 179 | from_addr, 180 | from_port: source.1, 181 | to_addr, 182 | to_port: target.1, 183 | }, 184 | delta: diff, 185 | } 186 | }); 187 | 188 | let messages = 189 | replayed 190 | .flat_map(move |(mut ts,_,x)| { 191 | if let Messages(event) = x { 192 | ts = round_duration_up_to_ns(ts, frequency_ns); 193 | Some((event.channel, ts, event.length as isize)) 194 | } 195 | else { 196 | None 197 | } 198 | }) 199 | .as_collection() 200 | .count() 201 | .inner 202 | .map(|((channel_id, total_count), _time, diff)| { 203 | Update { 204 | event: Event::Messages { 205 | channel_id, 206 | total_count, 207 | }, 208 | delta: diff, 209 | } 210 | }); 211 | 212 | let schedule = 213 | replayed 214 | .flat_map(move |(ts, worker, x)| if let Schedule(event) = x { Some((ts, worker, event)) } else { None }) 215 | .unary(timely::dataflow::channels::pact::Pipeline, "Schedules", |_,_| { 216 | 217 | let mut map = std::collections::HashMap::new(); 218 | let mut vec = Vec::new(); 219 | 220 | move |input, output| { 221 | 222 | input.for_each(|time, data| { 223 | data.swap(&mut vec); 224 | let mut session = output.session(&time); 225 | for (ts, worker, event) in vec.drain(..) { 226 | let key = (worker, event.id); 227 | match event.start_stop { 228 | timely::logging::StartStop::Start => { 229 | assert!(!map.contains_key(&key)); 230 | map.insert(key, ts); 231 | }, 232 | timely::logging::StartStop::Stop => { 233 | assert!(map.contains_key(&key)); 234 | let start = map.remove(&key).unwrap(); 235 | let mut ts_clip = round_duration_up_to_ns(ts, frequency_ns); 236 | let elapsed = ts - start; 237 | let elapsed_ns = (elapsed.as_secs() as isize) * 1_000_000_000 + (elapsed.subsec_nanos() as isize); 238 | session.give((key.1, ts_clip, elapsed_ns)); 239 | } 240 | } 241 | } 242 | }); 243 | } 244 | }) 245 | .as_collection() 246 | .count() 247 | .inner 248 | .map(|((operate_id, elapsed_ns), _time, diff)| { 249 | Update { 250 | event: Event::Schedule { 251 | operate_id, 252 | elapsed_ns, 253 | }, 254 | delta: diff, 255 | } 256 | }); 257 | 258 | operates 259 | .concat(&channels) 260 | .concat(&messages) 261 | .concat(&schedule) 262 | .sink(timely::dataflow::channels::pact::Pipeline, "ToVega", move |input| { 263 | 264 | let mut updates = Vec::new(); 265 | 266 | input.for_each(|_time, dataz| { 267 | for update in dataz.iter().cloned() { 268 | updates.push(update); 269 | } 270 | }); 271 | 272 | updates.sort(); 273 | for i in 1 .. updates.len() { 274 | if updates[i-1].event == updates[i].event { 275 | updates[i].delta += updates[i-1].delta; 276 | updates[i-1].delta = 0; 277 | } 278 | } 279 | updates.retain(|x| x.delta != 0); 280 | 281 | if !updates.is_empty() { 282 | for chan in endpoint.lock().unwrap().iter_mut() { 283 | let updates = Updates { updates: updates.clone() }; 284 | chan.send(updates).wait().unwrap(); 285 | } 286 | } 287 | }); 288 | }); 289 | 290 | }).unwrap(); // asserts error-free execution 291 | 292 | })); 293 | 294 | fn index(_req: HttpRequest) -> Result { 295 | let html_file = ::std::env::args().nth(3).expect("Must provide path to html file"); 296 | Ok(NamedFile::open(html_file)?) 297 | } 298 | 299 | server::new(move || { 300 | let endpoint_actix = endpoint_actix.clone(); 301 | App::new() 302 | .resource("/ws/", move |r| { 303 | let endpoint_actix = endpoint_actix.clone(); 304 | r.f(move |req| ws::start(req, Ws { addr: endpoint_actix.clone() })) 305 | }) 306 | .resource("/", |r| r.method(Method::GET).f(index)) 307 | }).bind("0.0.0.0:9000").unwrap().run(); 308 | } 309 | 310 | fn round_duration_up_to_ns(mut duration: Duration, nanos: u32) -> Duration { 311 | duration += Duration::from_nanos(nanos as u64); 312 | duration /= nanos; 313 | duration *= nanos; 314 | duration 315 | } 316 | -------------------------------------------------------------------------------- /src/bin/graph.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | extern crate timely_viz; 4 | 5 | extern crate abomonation; 6 | #[macro_use] extern crate abomonation_derive; 7 | 8 | extern crate actix; 9 | extern crate actix_web; 10 | extern crate futures; 11 | 12 | extern crate serde; 13 | #[macro_use] extern crate serde_derive; 14 | extern crate serde_json; 15 | 16 | use std::sync::{Arc, Mutex}; 17 | 18 | use timely::dataflow::operators::{Map, capture::Replay, Concat, Inspect}; 19 | use timely::logging::TimelyEvent::{Operates, Channels}; 20 | 21 | use differential_dataflow::collection::AsCollection; 22 | use differential_dataflow::operators::{Consolidate, Join, Threshold}; 23 | 24 | use actix::*; 25 | use actix_web::*; 26 | use actix_web::{App, HttpRequest, Result, http::Method, fs::NamedFile}; 27 | 28 | use futures::future::Future; 29 | 30 | #[derive(Serialize, Abomonation, Clone, PartialOrd, PartialEq, Eq, Ord, Debug)] 31 | enum Update { 32 | Operate { 33 | addr: Vec, 34 | name: String, 35 | }, 36 | Channel { 37 | id: usize, 38 | from_addr: Vec, 39 | to_addr: Vec, 40 | subgraph: bool, 41 | }, 42 | } 43 | 44 | // ==== Web-server setup ==== 45 | use timely_viz::server::Ws; 46 | 47 | #[derive(Serialize, Clone)] 48 | struct Updates { 49 | updates: Vec, 50 | } 51 | 52 | impl actix::Message for Update { 53 | type Result = (); 54 | } 55 | 56 | impl actix::Message for Updates { 57 | type Result = (); 58 | } 59 | 60 | impl Handler for Ws { 61 | type Result = (); 62 | 63 | fn handle(&mut self, msg: Updates, ctx: &mut Self::Context) { 64 | ctx.text(serde_json::to_string(&msg).unwrap()); 65 | } 66 | } 67 | // ========================== 68 | 69 | fn main() { 70 | let endpoint: Arc>>> = Arc::new(Mutex::new(Vec::new())); 71 | let endpoint_actix = endpoint.clone(); 72 | 73 | ::std::mem::drop(::std::thread::spawn(move || { 74 | 75 | let mut args = ::std::env::args(); 76 | args.next().unwrap(); 77 | 78 | // the number of workers in the computation we're examining 79 | let source_peers = args.next().unwrap().parse::().unwrap(); 80 | // one socket per worker in the computation we're examining 81 | let sockets = timely_viz::open_sockets(source_peers); 82 | 83 | timely::execute_from_args(args, move |worker| { 84 | let endpoint = endpoint.clone(); 85 | let sockets = sockets.clone(); 86 | 87 | // create replayers from disjoint partition of source worker identifiers. 88 | let replayers = timely_viz::make_replayers(sockets, worker.index(), worker.peers()); 89 | 90 | worker.dataflow(|scope| { 91 | let stream = 92 | replayers 93 | .replay_into(scope); 94 | 95 | let operates = 96 | stream 97 | .flat_map(|(t,_,x)| if let Operates(event) = x { Some((event, t, 1 as isize)) } else { None }) 98 | .as_collection(); 99 | 100 | let channels = 101 | stream 102 | .flat_map(|(t,_,x)| if let Channels(event) = x { Some((event, t, 1 as isize)) } else { None }) 103 | .as_collection(); 104 | 105 | // == Fix addresses so we can connect operators outside and inside subgraphs == 106 | 107 | let operates = operates.map(|event| (event.addr, event.name)).inspect(|x| println!("Operates: {:?}", x.0)); 108 | 109 | let operates_anti = operates.map(|(mut addr, _)| { 110 | addr.pop(); 111 | addr 112 | }); 113 | 114 | let operates_without_subg = operates.antijoin(&operates_anti.distinct()); 115 | 116 | let channels = channels.map( 117 | |event| (event.id, (event.scope_addr, event.source, event.target))).inspect(|x| println!("Channels: {:?}", x.0)); 118 | 119 | ({ 120 | operates_without_subg 121 | .filter(|(addr, _)| addr[0] == 0) 122 | .consolidate() 123 | .inner 124 | .map(move |((addr, name), _, _)| { 125 | let mut addr = addr.clone(); 126 | addr.remove(0); 127 | Update::Operate { 128 | addr: addr.clone(), 129 | name: name.clone(), 130 | } 131 | }) 132 | }).concat(&{ 133 | let worker_0 = channels 134 | .filter(|(_, (scope_addr, _, _))| scope_addr[0] == 0) 135 | .map(|(id, (scope_addr, from, to))| { 136 | let mut scope_addr = scope_addr.clone(); 137 | scope_addr.remove(0); 138 | (id, (scope_addr, from, to)) 139 | }); 140 | 141 | let subg_edges = worker_0 142 | .filter(|(_, (_, from, to))| from.0 == 0 || to.0 == 0) 143 | .flat_map(|(id, (scope_addr, from, to))| vec![ 144 | ((scope_addr.clone(), from.1), (id, (scope_addr.clone(), from, to))), 145 | ((scope_addr.clone(), to.1), (id, (scope_addr.clone(), from, to)))].into_iter()); 146 | 147 | let subg_incoming = subg_edges 148 | .join_map(&worker_0.map(|(id, (scope_addr, from, to))| { 149 | let mut from_addr = scope_addr.clone(); 150 | from_addr.push(from.0); 151 | ((from_addr, from.1), (id, (scope_addr, from, to))) 152 | }), |(_from_addr, _), (id, (scope_addr_from, from, _)), (_, (scope_addr_to, _, to))| { 153 | let mut from_addr = scope_addr_from.clone(); 154 | from_addr.push(from.0); 155 | let mut to_addr = scope_addr_to.clone(); 156 | to_addr.push(to.0); 157 | Update::Channel { 158 | id: *id, 159 | subgraph: true, 160 | from_addr, 161 | to_addr, 162 | } 163 | }); 164 | 165 | let subg_outgoing = subg_edges 166 | .join_map(&worker_0.map(|(id, (scope_addr, from, to))| { 167 | let mut to_addr = scope_addr.clone(); 168 | to_addr.push(to.0); 169 | ((to_addr, to.1), (id, (scope_addr, from, to))) 170 | }), |(_to_addr, _), (id, (scope_addr_to, _, to)), (_, (scope_addr_from, from, _))| { 171 | let mut from_addr = scope_addr_from.clone(); 172 | from_addr.push(from.0); 173 | let mut to_addr = scope_addr_to.clone(); 174 | to_addr.push(to.0); 175 | Update::Channel { 176 | id: *id, 177 | subgraph: true, 178 | from_addr, 179 | to_addr, 180 | } 181 | }); 182 | 183 | worker_0 184 | .filter(|(_, (_, from, to))| from.0 != 0 && to.0 != 0) 185 | .consolidate() 186 | .inner 187 | .map(move |((id, (scope_addr, (from, _), (to, _))), _, _)| { 188 | let mut from_addr = scope_addr.clone(); 189 | from_addr.push(from); 190 | let mut to_addr = scope_addr.clone(); 191 | to_addr.push(to); 192 | Update::Channel { 193 | id, 194 | subgraph: false, 195 | from_addr, 196 | to_addr, 197 | } 198 | }) 199 | .concat(&subg_incoming.inner.map(|(x, _, _)| x)) 200 | .concat(&subg_outgoing.inner.map(|(x, _, _)| x)) 201 | // Push the updates to the browser 202 | }).inspect_batch(move |_, x| { 203 | for chan in endpoint.lock().unwrap().iter_mut() { 204 | chan.send(Updates { updates: x.to_vec() }).wait().unwrap(); 205 | } 206 | }); 207 | }) 208 | }).unwrap(); // asserts error-free execution 209 | 210 | })); 211 | 212 | 213 | // == Start the web-server == 214 | fn index(_req: HttpRequest) -> Result { 215 | Ok(NamedFile::open("html/graph.html")?) 216 | } 217 | 218 | server::new(move || { 219 | let endpoint_actix = endpoint_actix.clone(); 220 | App::new() 221 | .resource("/ws/", move |r| { 222 | let endpoint_actix = endpoint_actix.clone(); 223 | r.f(move |req| ws::start(req, Ws { addr: endpoint_actix.clone() })) 224 | }) 225 | .resource("/", |r| r.method(Method::GET).f(index)) 226 | }).bind("0.0.0.0:9000").unwrap().disable_signals().run(); 227 | } 228 | -------------------------------------------------------------------------------- /src/bin/schedule.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | extern crate actix; 4 | extern crate actix_web; 5 | extern crate futures; 6 | extern crate timely_viz; 7 | 8 | extern crate serde; 9 | #[macro_use] extern crate serde_derive; 10 | extern crate serde_json; 11 | 12 | use std::time::Duration; 13 | use std::sync::{Arc, Mutex}; 14 | 15 | use timely::dataflow::operators::{Map, capture::Replay}; 16 | use timely::logging::TimelyEvent::{Operates, Schedule}; 17 | use timely::dataflow::operators::Operator; 18 | 19 | use differential_dataflow::AsCollection; 20 | use differential_dataflow::operators::{Join, Count}; 21 | 22 | use actix::*; 23 | use actix_web::*; 24 | use actix_web::{App, HttpRequest, Result, http::Method, fs::NamedFile}; 25 | 26 | use futures::future::Future; 27 | 28 | // ==== Web-server setup ==== 29 | use timely_viz::server::Ws; 30 | 31 | #[derive(Serialize, Clone, Ord, PartialOrd, Eq, PartialEq)] 32 | struct Update { 33 | data: String, // name of data source. 34 | name: String, 35 | id: String, 36 | parent_id: String, 37 | size: isize, 38 | diff: isize, 39 | } 40 | 41 | #[derive(Serialize, Clone)] 42 | struct Updates { 43 | updates: Vec, 44 | } 45 | 46 | impl actix::Message for Updates { 47 | type Result = (); 48 | } 49 | 50 | impl Handler for Ws{ 51 | type Result = (); 52 | 53 | fn handle(&mut self, msg: Updates, ctx: &mut Self::Context) { 54 | let text = serde_json::to_string(&msg).unwrap(); 55 | println!("Sending: {}", text); 56 | ctx.text(text); 57 | } 58 | } 59 | // ========================== 60 | 61 | fn main() { 62 | let endpoint: Arc>>> = Arc::new(Mutex::new(Vec::new())); 63 | let endpoint_actix = endpoint.clone(); 64 | 65 | ::std::mem::drop(::std::thread::spawn(move || { 66 | 67 | let mut args = ::std::env::args(); 68 | args.next().unwrap(); 69 | 70 | // the number of workers in the computation we're examining 71 | let source_peers = args.next().unwrap().parse::().unwrap(); 72 | let sockets = timely_viz::open_sockets(source_peers); 73 | 74 | timely::execute_from_args(std::env::args(), move |worker| { 75 | 76 | let endpoint = endpoint.clone(); 77 | 78 | let sockets = sockets.clone(); 79 | 80 | // create replayers from disjoint partition of source worker identifiers 81 | let replayers = timely_viz::make_replayers(sockets, worker.index(), worker.peers()); 82 | 83 | worker.dataflow(|scope| { 84 | 85 | let replayed = replayers.replay_into(scope); 86 | 87 | let operates = 88 | replayed 89 | .flat_map(|(ts, _setup, datum)| 90 | if let Operates(event) = datum { 91 | let ts = Duration::from_secs(ts.as_secs() + 1); 92 | Some((event, ts, 1)) 93 | } 94 | else { 95 | None 96 | } 97 | ) 98 | .as_collection(); 99 | 100 | let schedule = 101 | replayed 102 | .flat_map(|(ts, worker, x)| if let Schedule(event) = x { Some((ts, worker, event)) } 103 | else { None }) 104 | .unary(timely::dataflow::channels::pact::Pipeline, "Schedules", |_,_| { 105 | 106 | let mut map = std::collections::HashMap::new(); 107 | let mut vec = Vec::new(); 108 | 109 | move |input, output| { 110 | 111 | input.for_each(|time, data| { 112 | data.swap(&mut vec); 113 | let mut session = output.session(&time); 114 | for (ts, worker, event) in vec.drain(..) { 115 | let key = (worker, event.id); 116 | match event.start_stop { 117 | timely::logging::StartStop::Start => { 118 | assert!(!map.contains_key(&key)); 119 | map.insert(key, ts); 120 | }, 121 | timely::logging::StartStop::Stop => { 122 | assert!(map.contains_key(&key)); 123 | let end = map.remove(&key).unwrap(); 124 | // if work { 125 | let ts_clip = Duration::from_secs(ts.as_secs() + 1); 126 | let elapsed = ts - end; 127 | let elapsed_ns = (elapsed.as_secs() as isize) * 1_000_000_000 + (elapsed.subsec_nanos() as isize); 128 | session.give((key.1, ts_clip, elapsed_ns)); 129 | // } 130 | } 131 | } 132 | } 133 | }); 134 | } 135 | }) 136 | .as_collection(); 137 | 138 | operates 139 | .map(|x| (x.id, x.addr)) 140 | .filter(|x| x.1[0] == 0) 141 | .semijoin(&schedule) 142 | .map(|(_id, addr)| addr) 143 | .count() 144 | .inner 145 | .sink(timely::dataflow::channels::pact::Pipeline, "ToVega", move |input| { 146 | let mut updates = Updates { updates: Vec::new() }; 147 | 148 | input.for_each(|_time, dataz| { 149 | for ((ref addr, count), _time, diff) in dataz.iter() { 150 | if addr.len() > 0 { 151 | updates.updates.push( 152 | Update { 153 | data: "tree".to_string(), 154 | name: format!("{:?}", &addr[..]), 155 | id: format!("{:?}", &addr[..]), 156 | parent_id: format!("{:?}", &addr[.. addr.len()-1]), 157 | size: *count, 158 | diff: *diff, 159 | } 160 | ); 161 | } 162 | } 163 | }); 164 | 165 | updates.updates.sort(); 166 | for i in 1 .. updates.updates.len() { 167 | if updates.updates[i-1].id == updates.updates[i].id && updates.updates[i-1].size == updates.updates[i].size { 168 | updates.updates[i].diff += updates.updates[i-1].diff; 169 | updates.updates[i-1].diff = 0; 170 | } 171 | } 172 | updates.updates.retain(|x| x.diff != 0); 173 | if !updates.updates.is_empty() { 174 | for chan in endpoint.lock().unwrap().iter_mut() { 175 | chan.send(updates.clone()).wait().unwrap(); 176 | } 177 | } 178 | }); 179 | }); 180 | 181 | }).unwrap(); // asserts error-free execution 182 | })); 183 | 184 | fn index(_req: HttpRequest) -> Result { 185 | Ok(NamedFile::open("html/schedule.html")?) 186 | } 187 | 188 | server::new(move || { 189 | let endpoint_actix = endpoint_actix.clone(); 190 | App::new() 191 | .resource("/ws/", move |r| { 192 | let endpoint_actix = endpoint_actix.clone(); 193 | r.f(move |req| ws::start(req, Ws { addr: endpoint_actix.clone() })) 194 | }) 195 | .resource("/", |r| r.method(Method::GET).f(index)) 196 | }).bind("0.0.0.0:9000").unwrap().run(); 197 | } 198 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate differential_dataflow; 3 | 4 | extern crate actix; 5 | extern crate actix_web; 6 | 7 | pub mod server; 8 | 9 | use std::time::Duration; 10 | use std::sync::{Arc, Mutex}; 11 | use std::net::{TcpStream, TcpListener}; 12 | 13 | use timely::dataflow::operators::capture::EventReader; 14 | use timely::logging::{WorkerIdentifier, TimelyEvent}; 15 | 16 | /// Listens on 127.0.0.1:8000 and opens `source_peers` sockets from the 17 | /// computations we're examining. 18 | pub fn open_sockets(source_peers: usize) -> Arc>>> { 19 | let listener = TcpListener::bind("127.0.0.1:8000").unwrap(); 20 | Arc::new(Mutex::new( 21 | (0..source_peers).map(|_| { 22 | Some(listener.incoming().next().unwrap().unwrap()) 23 | }).collect::>())) 24 | } 25 | 26 | /// Construct replayers that read data from sockets and can stream it into 27 | /// timely dataflow. 28 | pub fn make_replayers(sockets: Arc>>>, index: usize, peers: usize) -> Vec> { 29 | 30 | sockets.lock().unwrap() 31 | .iter_mut().enumerate() 32 | .filter(|(i, _)| *i % peers == index) 33 | .map(move |(_, s)| s.take().unwrap()) 34 | .map(|r| EventReader::::new(r)) 35 | .collect::>() 36 | } 37 | -------------------------------------------------------------------------------- /src/server.rs: -------------------------------------------------------------------------------- 1 | /// Web-server 2 | 3 | use actix::*; 4 | use actix_web::*; 5 | 6 | use std::sync::{Arc, Mutex}; 7 | 8 | pub struct Ws { 9 | pub addr: Arc>>>, 10 | } 11 | 12 | impl Actor for Ws { 13 | type Context = ws::WebsocketContext; 14 | 15 | fn started(&mut self, ctx: &mut Self::Context) { 16 | let addr: Addr = ctx.address(); 17 | self.addr.lock().unwrap().push(addr); 18 | } 19 | } 20 | 21 | impl StreamHandler for Ws { 22 | fn handle(&mut self, msg: ws::Message, ctx: &mut Self::Context) { 23 | match msg { 24 | ws::Message::Ping(msg) => ctx.pong(&msg), 25 | _ => (), 26 | } 27 | } 28 | } 29 | --------------------------------------------------------------------------------