Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions mdbook/src/SUMMARY.md
Original file line number Diff line number Diff line change
Expand Up @@ -36,4 +36,5 @@
- [Internals](./chapter_5/chapter_5.md)
- [Communication](./chapter_5/chapter_5_1.md)
- [Progress Tracking](./chapter_5/chapter_5_2.md)
- [Containers](./chapter_5/chapter_5_3.md)
- [Logging](./chapter_5/chapter_5_4.md)
385 changes: 200 additions & 185 deletions mdbook/src/chapter_5/chapter_5_4.md

Large diffs are not rendered by default.

6 changes: 6 additions & 0 deletions timely/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ workspace = true
[features]
default = ["getopts"]
getopts = ["dep:getopts", "timely_communication/getopts"]
visualizer = ["dep:tungstenite"]

[dependencies]
columnar = { workspace = true }
Expand All @@ -35,3 +36,8 @@ timely_logging = { path = "../logging", version = "0.27" }
timely_communication = { path = "../communication", version = "0.27", default-features = false }
timely_container = { path = "../container", version = "0.27" }
smallvec = { version = "1.15.1", features = ["serde", "const_generics"] }
tungstenite = { version = "0.26", optional = true }

[[example]]
name = "logging-live"
required-features = ["visualizer"]
77 changes: 77 additions & 0 deletions timely/examples/logging-live.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
//! Streams timely events over WebSocket for live visualization.
//!
//! Usage:
//! cargo run --example logging-live --features visualizer [-- timely args]
//!
//! Then open `visualizer/index.html` in a browser and connect to `ws://localhost:51371`.

use timely::dataflow::operators::{Input, Exchange, Enter, Leave, Inspect, Probe, Feedback, ConnectLoop, Concat};
use timely::dataflow::operators::vec::{Map, Filter};
use timely::dataflow::{InputHandle, Scope};
use timely::visualizer::Server;

fn main() {
let port = std::env::var("TIMELY_VIS_PORT")
.ok()
.and_then(|s| s.parse().ok())
.unwrap_or(51371);

let server = Server::start(port);

timely::execute_from_args(std::env::args(), move |worker| {

server.register(worker);

// A richer dataflow to exercise the visualizer:
//
// Input -> Map -> Exchange -> Filter ----> Region[ Inspect ] -> Probe
// \ ^
// +-> Feedback ----+
//
// This gives us: multiple pipeline stages, an exchange, branching,
// a nested scope (region), a feedback loop with a back-edge, and
// real data flowing through channels.
let mut input = InputHandle::new();
let mut probe = timely::dataflow::ProbeHandle::new();
worker.dataflow(|scope| {
let (handle, cycle) = scope.feedback::<Vec<u64>>(1);

let mapped = scope
.input_from(&mut input)
.container::<Vec<_>>()
.map(|x: u64| x.wrapping_mul(17).wrapping_add(3))
.exchange(|&x: &u64| x);

let filtered = mapped.filter(|&x: &u64| x % 2 == 0);

let looped = filtered.clone().concat(cycle);

scope.region(|inner| {
looped
.enter(inner)
.inspect(|_x| { })
.leave()
})
.probe_with(&mut probe);

// Feed back values that haven't reached zero yet.
filtered
.map(|x: u64| x / 4)
.filter(|&x: &u64| x > 0)
.connect_loop(handle);
});

// Continuously feed data so events keep flowing.
let mut round = 0u64;
loop {
for i in 0..100u64 {
input.send(round * 100 + i);
}
round += 1;
input.advance_to(round);
while probe.less_than(input.time()) {
worker.step();
}
}
}).unwrap();
}
3 changes: 3 additions & 0 deletions timely/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,9 @@ pub mod order;
pub mod logging;
// pub mod log_events;

#[cfg(feature = "visualizer")]
pub mod visualizer;

pub mod scheduling;

/// A composite trait for types usable as containers in timely dataflow.
Expand Down
4 changes: 2 additions & 2 deletions timely/src/logging.rs
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ impl<P, C> Drop for BatchLogger<P, C> where P: EventPusher<Duration, C> {
pub struct OperatesEvent {
/// Worker-unique identifier for the operator.
pub id: usize,
/// Sequence of nested scope identifiers indicating the path from the root to the scope containing this operator.
/// Sequence of nested scope identifiers indicating the path from the root to this operator.
pub addr: Vec<usize>,
/// A helpful name.
pub name: String,
Expand All @@ -85,7 +85,7 @@ pub struct OperatesSummaryEvent<TS> {
pub struct ChannelsEvent {
/// Worker-unique identifier for the channel
pub id: usize,
/// Sequence of nested scope identifiers indicating the path from the root to this instance.
/// Sequence of nested scope identifiers indicating the path from the root to the scope containing this channel.
pub scope_addr: Vec<usize>,
/// Source descriptor, indicating operator index and output port.
pub source: (usize, usize),
Expand Down
6 changes: 5 additions & 1 deletion timely/src/progress/subgraph.rs
Original file line number Diff line number Diff line change
Expand Up @@ -136,9 +136,13 @@
pub fn add_child(&mut self, child: Box<dyn Operate<TInner>>, index: usize, identifier: usize) {
let child = PerOperatorState::new(child, index, identifier, self.logging.clone(), &mut self.summary_logging);
if let Some(l) = &mut self.logging {
let mut child_path = Vec::with_capacity(self.path.len() + 1);
child_path.extend_from_slice(&self.path[..]);
child_path.push(index);

l.log(crate::logging::OperatesEvent {
id: identifier,
addr: self.path.to_vec(),
addr: child_path,
name: child.name.to_owned(),
});
}
Expand Down Expand Up @@ -790,7 +794,7 @@
for (time, diff) in internal.iter() {
if *diff > 0 {
let consumed = shared_progress.consumeds.iter_mut().any(|x| x.iter().any(|(t,d)| *d > 0 && t.less_equal(time)));
let internal = child_state.sources[output].implications.less_equal(time);

Check warning on line 797 in timely/src/progress/subgraph.rs

View workflow job for this annotation

GitHub Actions / Cargo clippy

`internal` shadows a previous, unrelated binding
if !consumed && !internal {
println!("Increment at {:?}, not supported by\n\tconsumed: {:?}\n\tinternal: {:?}", time, shared_progress.consumeds, child_state.sources[output].implications);
panic!("Progress error; internal {:?}", self.name);
Expand Down
232 changes: 232 additions & 0 deletions timely/src/visualizer.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,232 @@
//! Live dataflow visualization over WebSocket.
//!
//! This module provides a [`Server`] that streams timely logging events as JSON
//! over WebSocket to the browser-based visualizer (`visualizer/index.html`).
//!
//! # Usage
//!
//! ```ignore
//! use timely::visualizer::Server;
//!
//! let server = Server::start(51371);
//!
//! timely::execute_from_args(std::env::args(), move |worker| {
//! server.register(worker);
//! // ... build and run dataflows ...
//! }).unwrap();
//! ```
//!
//! Then open `visualizer/index.html` in a browser and connect to `ws://localhost:51371`.
//!
//! Requires the `visualizer` feature flag.

use std::net::{TcpListener, TcpStream};
use std::sync::{Arc, Mutex, mpsc};
use std::thread;
use std::time::{Duration, Instant};

use tungstenite::{accept, Message};

use crate::logging::{TimelyEvent, TimelyEventBuilder, StartStop};
use crate::worker::Worker;
use crate::communication::Allocate;

/// A handle to the visualization WebSocket server.
///
/// The server accepts browser connections and streams timely events as JSON.
/// Structural events (`Operates`, `Channels`) are replayed to late-connecting
/// clients so they can reconstruct the dataflow graph.
///
/// `Server` is cheaply cloneable — all clones share the same underlying channel
/// and server thread. The server thread runs until all `Server` handles (and
/// their associated senders) are dropped.
#[derive(Clone)]
pub struct Server {
tx: Arc<Mutex<mpsc::Sender<String>>>,
}

impl Server {
/// Start the WebSocket visualization server on the given port.
///
/// Spawns a background thread that accepts WebSocket connections and
/// broadcasts events. The thread exits when all `Server` handles are
/// dropped.
///
/// # Panics
///
/// Panics if the port cannot be bound.
pub fn start(port: u16) -> Self {
let (tx, rx) = mpsc::channel::<String>();

thread::spawn(move || run_ws_server(port, rx));

eprintln!("Visualizer WebSocket server on ws://localhost:{port}");
eprintln!("Open visualizer/index.html and connect to the address above.");

Server {
tx: Arc::new(Mutex::new(tx)),
}
}

/// Register the timely event logger for this worker.
///
/// This installs a logging callback on the `"timely"` log stream that
/// serializes events as JSON and sends them to the WebSocket server.
pub fn register<A: Allocate>(&self, worker: &mut Worker<A>) {
let tx = Arc::clone(&self.tx);
let worker_index = worker.index();

worker.log_register().unwrap().insert::<TimelyEventBuilder, _>(
"timely",
move |_time, data| {
if let Some(data) = data {
let tx = tx.lock().unwrap();
for (elapsed, event) in data.iter() {
let duration_ns =
elapsed.as_secs() as u64 * 1_000_000_000
+ elapsed.subsec_nanos() as u64;
if let Some(json) = event_to_json(worker_index, duration_ns, event) {
let _ = tx.send(json);
}
}
}
},
);
}
}

/// Convert a timely event to a JSON string: `[worker, duration_ns, event]`.
fn event_to_json(worker: usize, duration_ns: u64, event: &TimelyEvent) -> Option<String> {
let event_json = match event {
TimelyEvent::Operates(e) => {
let addr: Vec<String> = e.addr.iter().map(|a| a.to_string()).collect();
let name = e.name.replace('\\', "\\\\").replace('"', "\\\"");
format!(
r#"{{"Operates": {{"id": {}, "addr": [{}], "name": "{}"}}}}"#,
e.id, addr.join(", "), name,
)
}
TimelyEvent::Channels(e) => {
let scope_addr: Vec<String> = e.scope_addr.iter().map(|a| a.to_string()).collect();
let typ = e.typ.replace('\\', "\\\\").replace('"', "\\\"");
format!(
r#"{{"Channels": {{"id": {}, "scope_addr": [{}], "source": [{}, {}], "target": [{}, {}], "typ": "{}"}}}}"#,
e.id, scope_addr.join(", "),
e.source.0, e.source.1, e.target.0, e.target.1, typ,
)
}
TimelyEvent::Schedule(e) => {
let ss = match e.start_stop {
StartStop::Start => "\"Start\"",
StartStop::Stop => "\"Stop\"",
};
format!(
r#"{{"Schedule": {{"id": {}, "start_stop": {}}}}}"#,
e.id, ss,
)
}
TimelyEvent::Messages(e) => {
format!(
r#"{{"Messages": {{"is_send": {}, "channel": {}, "source": {}, "target": {}, "seq_no": {}, "record_count": {}}}}}"#,
e.is_send, e.channel, e.source, e.target, e.seq_no, e.record_count,
)
}
TimelyEvent::Shutdown(e) => {
format!(r#"{{"Shutdown": {{"id": {}}}}}"#, e.id)
}
_ => return None,
};
Some(format!("[{worker}, {duration_ns}, {event_json}]"))
}

const FLUSH_INTERVAL: Duration = Duration::from_millis(100);

/// Run the WebSocket server. Batches events and replays structural events to
/// late-connecting clients.
fn run_ws_server(port: u16, rx: mpsc::Receiver<String>) {
let listener = TcpListener::bind(format!("0.0.0.0:{port}"))
.unwrap_or_else(|e| panic!("Failed to bind to port {port}: {e}"));
listener.set_nonblocking(true).expect("Cannot set non-blocking");

let mut clients: Vec<tungstenite::WebSocket<TcpStream>> = Vec::new();
let mut batch: Vec<String> = Vec::new();
let mut replay: Vec<String> = Vec::new();
let mut done = false;

loop {
let deadline = Instant::now() + FLUSH_INTERVAL;

// Accept pending connections (non-blocking).
loop {
match listener.accept() {
Ok((stream, addr)) => {
eprintln!("Visualizer client connected from {addr}");
stream.set_nonblocking(false).ok();
match accept(stream) {
Ok(mut ws) => {
if !replay.is_empty() {
let payload = format!("[{}]", replay.join(","));
let msg = Message::Text(payload.into());
if ws.send(msg).is_err() {
eprintln!("Failed to send replay; dropping client");
continue;
}
}
clients.push(ws);
}
Err(e) => eprintln!("WebSocket handshake failed: {e}"),
}
}
Err(ref e) if e.kind() == std::io::ErrorKind::WouldBlock => break,
Err(e) => {
eprintln!("Accept error: {e}");
break;
}
}
}

// Drain the channel until the flush deadline.
loop {
let remaining = deadline.saturating_duration_since(Instant::now());
match rx.recv_timeout(remaining) {
Ok(json) => {
if json.contains("\"Operates\"") || json.contains("\"Channels\"") {
replay.push(json.clone());
}
batch.push(json);
}
Err(mpsc::RecvTimeoutError::Timeout) => break,
Err(mpsc::RecvTimeoutError::Disconnected) => { done = true; break; }
}
}

// Flush batch to all connected clients.
if !batch.is_empty() && !clients.is_empty() {
let payload = format!("[{}]", batch.join(","));
let msg = Message::Text(payload.into());
clients.retain_mut(|ws| {
match ws.send(msg.clone()) {
Ok(_) => true,
Err(_) => {
eprintln!("Visualizer client disconnected");
false
}
}
});
}
batch.clear();

if done { break; }
}

// Close all clients gracefully.
for ws in clients.iter_mut() {
let _ = ws.close(None);
loop {
match ws.read() {
Ok(_) => continue,
Err(_) => break,
}
}
}
}
2 changes: 1 addition & 1 deletion timely/src/worker.rs
Original file line number Diff line number Diff line change
Expand Up @@ -673,7 +673,7 @@ impl<A: Allocate> Worker<A> {
if let Some(l) = logging.as_mut() {
l.log(crate::logging::OperatesEvent {
id: identifier,
addr: vec![],
addr: operator.path().to_vec(),
name: operator.name().to_string(),
});
l.flush();
Expand Down
Loading
Loading