diff --git a/Cargo.toml b/Cargo.toml index e0b811ad9..08a7989e9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -13,6 +13,7 @@ members = [ #"tpchlike", #"doop", "mdbook", + "diagnostics", ] resolver = "2" @@ -22,8 +23,8 @@ rust-version = "1.86" [workspace.dependencies] differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.20.0" } -timely = { version = "0.27", default-features = false } -columnar = { version = "0.11", default-features = false } +timely = { version = "0.28", default-features = false } +columnar = { version = "0.12", default-features = false } #timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } #timely = { path = "../timely-dataflow/timely/", default-features = false } diff --git a/diagnostics/Cargo.toml b/diagnostics/Cargo.toml new file mode 100644 index 000000000..c4a4f89a3 --- /dev/null +++ b/diagnostics/Cargo.toml @@ -0,0 +1,12 @@ +[package] +name = "diagnostics" +version = "0.1.0" +edition.workspace = true +rust-version.workspace = true + +[dependencies] +differential-dataflow.workspace = true +timely = { workspace = true, features = ["getopts"] } +serde = { version = "1", features = ["derive"] } +serde_json = "1" +tungstenite = "0.26" diff --git a/diagnostics/examples/scc-bench.rs b/diagnostics/examples/scc-bench.rs new file mode 100644 index 000000000..549562ce8 --- /dev/null +++ b/diagnostics/examples/scc-bench.rs @@ -0,0 +1,126 @@ +//! SCC benchmark with diagnostics. +//! +//! Usage: scc-bench [timely args] [--log-logging] [nodes [edges [batch [rounds]]]] +//! +//! Supports standard timely arguments like `-w4` for 4 workers. With multiple +//! workers, all workers' diagnostics are exchanged to worker 0 for serving. +//! +//! Start this, then serve the diagnostics UI and open it in a browser: +//! +//! cd diagnostics && python3 -m http.server 8000 +//! open http://localhost:8000/index.html +//! +//! Click Connect (defaults to ws://localhost:51371). +//! +//! Pass --log-logging to also see the diagnostics dataflow itself. + +use std::hash::{Hash, Hasher}; +use std::collections::hash_map::DefaultHasher; + +use timely::dataflow::operators::probe::Handle; + +use differential_dataflow::input::Input; +use differential_dataflow::algorithms::graphs::scc::strongly_connected; + +use diagnostics::logging; +use diagnostics::server::Server; + +fn hash_to_u64(value: &T) -> u64 { + let mut hasher = DefaultHasher::new(); + value.hash(&mut hasher); + hasher.finish() +} + +fn edge_for(index: usize, nodes: usize) -> (usize, usize) { + let h1 = hash_to_u64(&index); + let h2 = hash_to_u64(&h1); + ((h1 as usize) % nodes, (h2 as usize) % nodes) +} + +fn main() { + let timer = std::time::Instant::now(); + + // Extract our flags before timely consumes its args. + let log_logging = std::env::args().any(|a| a == "--log-logging"); + + // timely::execute_from_args handles -w, -n, -p, -h and passes the rest through. + timely::execute_from_args(std::env::args(), move |worker| { + // Register diagnostics on each worker. + let state = logging::register(worker, log_logging); + + // Start the WebSocket server on worker 0 only. With multiple + // workers (-w N), all workers' data is exchanged to worker 0 + // via the DD arrangements, so the browser sees everything. + // + // Non-server workers must drop the SinkHandle so its BatchLogger + // sends a capability retraction and the client input Replay can + // advance. Without this, the Replay holds its frontier at time 0 + // forever, blocking the cross-join from producing output. + let _server = if worker.index() == 0 { + Some(Server::start(51371, state.sink)) + } else { + drop(state.sink); + None + }; + + // Parse positional args (skip flags consumed by timely and ourselves). + let positional: Vec = std::env::args() + .skip(1) + .filter(|a| !a.starts_with('-')) + .collect(); + let nodes: usize = positional.get(0).and_then(|s| s.parse().ok()).unwrap_or(100_000); + let edges: usize = positional.get(1).and_then(|s| s.parse().ok()).unwrap_or(200_000); + let batch: usize = positional.get(2).and_then(|s| s.parse().ok()).unwrap_or(1_000); + let rounds: usize = positional.get(3).and_then(|s| s.parse().ok()).unwrap_or(usize::MAX); + + if worker.index() == 0 { + println!("nodes: {nodes}, edges: {edges}, batch: {batch}, rounds: {}, workers: {}", + if rounds == usize::MAX { "∞".to_string() } else { rounds.to_string() }, + worker.peers()); + } + + let mut probe = Handle::new(); + let mut input = worker.dataflow(|scope| { + let (input, graph) = scope.new_collection::<(usize, usize), isize>(); + let _scc = strongly_connected(graph).probe_with(&mut probe); + input + }); + + let index = worker.index(); + let peers = worker.peers(); + + // Load initial edges (partitioned across workers). + let timer_load = std::time::Instant::now(); + for i in (0..edges).filter(|i| i % peers == index) { + input.insert(edge_for(i, nodes)); + } + input.advance_to(1); + input.flush(); + while probe.less_than(input.time()) { + worker.step(); + } + if index == 0 { + println!("{:?}\t{:?}\tloaded {edges} edges", timer.elapsed(), timer_load.elapsed()); + } + + // Apply changes in rounds. + for round in 0..rounds { + let timer_round = std::time::Instant::now(); + for i in (0..batch).filter(|i| i % peers == index) { + input.remove(edge_for(round * batch + i, nodes)); + input.insert(edge_for(edges + round * batch + i, nodes)); + } + input.advance_to(round + 2); + input.flush(); + while probe.less_than(input.time()) { + worker.step(); + } + if index == 0 { + println!("{:?}\t{:?}\tround {round} ({} changes)", + timer.elapsed(), timer_round.elapsed(), batch * 2); + } + } + }).unwrap(); + + println!("{:?}\tshut down", timer.elapsed()); +} diff --git a/diagnostics/examples/smoke.rs b/diagnostics/examples/smoke.rs new file mode 100644 index 000000000..d63937545 --- /dev/null +++ b/diagnostics/examples/smoke.rs @@ -0,0 +1,96 @@ +//! Smoke test: run a small DD computation with diagnostics and a WS server. +//! +//! Start this, then open a browser console and connect: +//! let ws = new WebSocket("ws://localhost:51371"); +//! ws.onmessage = e => console.log(JSON.parse(e.data)); +//! +//! You should see operator, channel, and stat updates flowing. + +use std::hash::{Hash, Hasher}; +use std::collections::hash_map::DefaultHasher; +use std::time::Duration; + +use timely::dataflow::operators::probe::Handle; +use differential_dataflow::input::Input; + +use diagnostics::logging; +use diagnostics::server::Server; + +fn hash_to_u64(value: &T) -> u64 { + let mut hasher = DefaultHasher::new(); + value.hash(&mut hasher); + hasher.finish() +} + +fn edge_for(index: usize, nodes: usize) -> (usize, usize) { + let h1 = hash_to_u64(&index); + let h2 = hash_to_u64(&h1); + ((h1 as usize) % nodes, (h2 as usize) % nodes) +} + +fn main() { + let timer = std::time::Instant::now(); + + timely::execute(timely::Config::thread(), move |worker| { + // Register diagnostics (log_logging = true to see the diagnostics dataflow itself). + let state = logging::register(worker, true); + + // Start the WebSocket server on worker 0 only. + // Non-server workers drop the SinkHandle so the client input + // Replay can advance its frontier. + let _server = if worker.index() == 0 { + Some(Server::start(51371, state.sink)) + } else { + drop(state.sink); + None + }; + + // Build a user dataflow. + let nodes = 1000; + let edges = 2000; + let batch = 100; + + let mut probe = Handle::new(); + let mut input = worker.dataflow(|scope| { + let (input, graph) = scope.new_collection::<(usize, usize), isize>(); + graph + .map(|(src, _dst)| src) + .probe_with(&mut probe); + input + }); + + // Load initial edges. + for i in 0..edges { + input.insert(edge_for(i, nodes)); + } + input.advance_to(1); + input.flush(); + while probe.less_than(input.time()) { + worker.step(); + } + if worker.index() == 0 { + eprintln!("{:?}\tloaded {edges} edges", timer.elapsed()); + } + + // Run rounds of changes, keeping the server alive for browsers to connect. + for round in 0..usize::MAX { + let round_timer = std::time::Instant::now(); + for i in 0..batch { + input.remove(edge_for(round * batch + i, nodes)); + input.insert(edge_for(edges + round * batch + i, nodes)); + } + input.advance_to(round + 2); + input.flush(); + while probe.less_than(input.time()) { + worker.step(); + } + if worker.index() == 0 && round % 100 == 0 { + eprintln!("{:?}\t{:?}\tround {round}", timer.elapsed(), round_timer.elapsed()); + } + + // Slow down so there's time to connect a browser. + std::thread::sleep(Duration::from_millis(10)); + } + }) + .unwrap(); +} diff --git a/diagnostics/index.html b/diagnostics/index.html new file mode 100644 index 000000000..403e39b2d --- /dev/null +++ b/diagnostics/index.html @@ -0,0 +1,1145 @@ + + + + + + Differential Dataflow Diagnostics + + + + + + +
+
+ + +
+
+ + + +
+ + + + diff --git a/diagnostics/src/lib.rs b/diagnostics/src/lib.rs new file mode 100644 index 000000000..1f61ba037 --- /dev/null +++ b/diagnostics/src/lib.rs @@ -0,0 +1,62 @@ +//! Diagnostics and visualization for timely and differential dataflow. +//! +//! This crate provides a live diagnostics console for timely and differential +//! dataflow computations. It captures logging events, maintains them as +//! differential dataflow collections with indexed arrangements, and serves +//! them to browser clients over WebSocket. +//! +//! # Quick start +//! +//! ```ignore +//! use diagnostics::{logging, server::Server}; +//! +//! timely::execute(config, |worker| { +//! // Register diagnostics logging on each worker. +//! let state = logging::register(worker, false); +//! +//! // Start the WebSocket server on worker 0 only. +//! // Other workers drop their sink handles (diagnostics collections +//! // are still maintained, but not served). +//! let _server = if worker.index() == 0 { +//! Some(Server::start(51371, state.sink)) +//! } else { +//! None +//! }; +//! +//! // Build your dataflow as usual... +//! // worker.dataflow(|scope| { ... }); +//! }); +//! ``` +//! +//! To view the diagnostics, serve the included `index.html` over HTTP +//! (browsers restrict WebSocket connections from `file://` URLs): +//! +//! ```text +//! cd diagnostics && python3 -m http.server 8000 +//! ``` +//! +//! Then open `http://localhost:8000/index.html` and click Connect. +//! The browser connects to `ws://localhost:51371` for live data. +//! +//! With multiple workers (`-w N`), all workers' diagnostics are exchanged +//! to worker 0 via the DD arrangements, so the browser sees everything. +//! +//! # Architecture +//! +//! [`logging::register`] builds a dataflow that: +//! 1. Captures timely and differential logging events via [`EventLink`](timely::dataflow::operators::capture::EventLink) pairs. +//! 2. Demuxes them into typed DD collections (operators, channels, schedule elapsed, message counts, arrangement stats). +//! 3. Arranges them into indexed traces for persistence. +//! 4. Cross-joins all collections with a client input: when a browser connects, the join naturally produces the full current state as a batch of diffs, followed by incremental updates. +//! 5. Captures the output into an `mpsc` channel for the WebSocket thread. +//! +//! [`server::Server`] runs a WebSocket server that: +//! - Accepts browser connections and assigns client IDs. +//! - Announces connect/disconnect to the dataflow via the client input channel. +//! - Reads diagnostic updates from the capture channel and routes them as JSON to the appropriate browser client. +//! +//! The browser-side `index.html` simply applies the diffs to maintain local +//! state and renders the dataflow graph — no client-side aggregation needed. + +pub mod logging; +pub mod server; diff --git a/diagnostics/src/logging.rs b/diagnostics/src/logging.rs new file mode 100644 index 000000000..b47721e62 --- /dev/null +++ b/diagnostics/src/logging.rs @@ -0,0 +1,662 @@ +//! Logging dataflows for timely and differential dataflow events. +//! +//! Captures timely and differential logging events, replays them as timely +//! streams, and maintains them as differential dataflow collections with +//! indexed arrangements and a client-driven sink. +//! +//! # Architecture +//! +//! The [`register`] function: +//! 1. Creates `EventLink` pairs for timely and DD event capture. +//! 2. Builds a dataflow that replays events into DD collections and arranges them. +//! 3. Creates a cross-join with a client input collection: when a client appears +//! at +1, the join naturally replays the full current state as updates. +//! 4. Captures the joined output via `mpsc` for the WebSocket thread. +//! 5. Registers logging callbacks that push events into the links. +//! +//! The WebSocket thread communicates client connect/disconnect via a +//! [`ClientInput`] that pushes through an `mpsc` channel, and reads +//! diagnostic updates from another `mpsc` channel. +//! +//! Timestamps use `Duration` (matching timely's logging infrastructure). +//! Timestamp quantization (rounding to interval boundaries) is done as a DD +//! `map_in_place` operation on the collections, keeping the logging layer simple. + +use std::borrow::Cow; +use std::collections::BTreeMap; +use std::rc::Rc; +use std::sync::mpsc; +use std::time::{Duration, Instant}; + +use differential_dataflow::collection::concatenate; +use differential_dataflow::logging::{DifferentialEvent, DifferentialEventBuilder}; +use differential_dataflow::operators::arrange::TraceAgent; +use differential_dataflow::trace::implementations::{KeySpine, ValSpine}; +use differential_dataflow::{AsCollection, VecCollection}; + +use timely::communication::Allocate; +use timely::container::CapacityContainerBuilder; +use timely::dataflow::channels::pact::Pipeline; +use timely::dataflow::operators::capture::{Event, EventLink, Replay, Capture}; +use timely::dataflow::operators::Exchange; +use timely::dataflow::operators::vec::Map; +use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; +use timely::dataflow::operators::generic::OutputBuilder; +use timely::dataflow::{Scope, Stream}; +use timely::logging::{ + BatchLogger, OperatesEvent, StartStop, TimelyEvent, TimelyEventBuilder, +}; +use timely::worker::Worker; + +use serde::{Serialize, Deserialize}; + +// ============================================================================ +// ClientInput — manages client connect/disconnect events across threads +// ============================================================================ + +/// Container type for client connection events: `(client_id, time, diff)`. +type ClientContainer = Vec<(usize, Duration, i64)>; + +/// Sends client connect/disconnect events to the diagnostics dataflow. +/// +/// The WebSocket server thread uses this to announce clients. On drop, +/// sends a capability retraction so the dataflow's client input frontier +/// can advance (important for multi-worker setups where non-server workers +/// must release their frontier). +pub struct ClientInput { + sender: mpsc::Sender>, + time: Duration, +} + +impl ClientInput { + /// Announce a client connection. + pub fn connect(&mut self, client_id: usize, elapsed: Duration) { + let _ = self + .sender + .send(Event::Messages(self.time, vec![(client_id, elapsed, 1)])); + self.advance(elapsed); + } + + /// Announce a client disconnection. + pub fn disconnect(&mut self, client_id: usize, elapsed: Duration) { + let _ = self + .sender + .send(Event::Messages(self.time, vec![(client_id, elapsed, -1)])); + self.advance(elapsed); + } + + /// Advance the capability to `elapsed`. Call periodically so the + /// dataflow's frontier can progress. + pub fn advance(&mut self, elapsed: Duration) { + if self.time < elapsed { + let _ = self + .sender + .send(Event::Progress(vec![(elapsed, 1), (self.time, -1)])); + self.time = elapsed; + } + } +} + +impl Drop for ClientInput { + fn drop(&mut self) { + let _ = self + .sender + .send(Event::Progress(vec![(self.time, -1)])); + } +} + +// ============================================================================ +// MpscEventIterator — bridges mpsc::Receiver to timely's EventIterator +// ============================================================================ + +/// Wraps an `mpsc::Receiver` as an `EventIterator` for use with `Replay`. +struct MpscEventIterator { + receiver: mpsc::Receiver>, +} + +impl timely::dataflow::operators::capture::event::EventIterator + for MpscEventIterator +{ + fn next(&mut self) -> Option>> { + self.receiver.try_recv().ok().map(Cow::Owned) + } +} + +// ============================================================================ +// Diagnostic update types +// ============================================================================ + +/// Identifies the kind of a key-only statistic (diff carries the value). +#[derive(Clone, Debug, Ord, PartialOrd, Eq, PartialEq, Hash, Serialize, Deserialize)] +pub enum StatKind { + Elapsed, + Messages, + ArrangementBatches, + ArrangementRecords, + Sharing, + BatcherRecords, + BatcherSize, + BatcherCapacity, + BatcherAllocations, +} + +/// A tagged diagnostic update sent to the WebSocket thread. +/// +/// Each variant carries enough information for the browser to apply the update. +/// The diff on the containing `(D, time, diff)` triple carries the magnitude. +#[derive(Clone, Debug, Ord, PartialOrd, Eq, PartialEq, Hash, Serialize, Deserialize)] +pub enum DiagnosticUpdate { + /// Operator appeared (+1) or disappeared (-1). Diff is ±1. + Operator { id: usize, name: String, addr: Vec }, + /// Channel appeared (+1). Diff is ±1. + Channel { id: usize, scope_addr: Vec, source: (usize, usize), target: (usize, usize) }, + /// A key-only statistic keyed by operator/channel id. Diff carries the value. + Stat { kind: StatKind, id: usize }, +} + +/// The container type for captured diagnostic output. +/// Each element is `((client_id, update), time, diff)`. +pub type DiagnosticContainer = Vec<((usize, DiagnosticUpdate), Duration, i64)>; + +/// The event type received by the WebSocket thread. +pub type DiagnosticEvent = Event; + +// ============================================================================ +// Trace handle types +// ============================================================================ + +/// A key-value trace: key K, value V, time Duration, diff i64. +type ValTrace = TraceAgent>; +/// A key-only trace: key K, time Duration, diff i64. +type KeyTrace = TraceAgent>; + +/// Trace handles for timely logging arrangements. +pub struct TimelyTraces { + /// Live operators arranged by id → (name, addr). + pub operators: ValTrace)>, + /// Live channels arranged by id → (scope_addr, source, target). + pub channels: ValTrace, (usize, usize), (usize, usize))>, + /// Schedule elapsed per operator (diff = nanoseconds). + pub elapsed: KeyTrace, + /// Message records sent per channel (diff = record count). + pub messages: KeyTrace, +} + +/// Trace handles for differential dataflow logging arrangements. +pub struct DifferentialTraces { + pub arrangement_batches: KeyTrace, + pub arrangement_records: KeyTrace, + pub sharing: KeyTrace, + pub batcher_records: KeyTrace, + pub batcher_size: KeyTrace, + pub batcher_capacity: KeyTrace, + pub batcher_allocations: KeyTrace, +} + +// ============================================================================ +// SinkHandle — returned to the caller for WebSocket integration +// ============================================================================ + +/// Handle for the WebSocket thread to interact with the diagnostics dataflow. +/// +/// **Important:** The WS thread must call `client_input.advance(elapsed)` +/// periodically (e.g., every 100ms–1s) to advance the client input's frontier. +/// Without this, the cross-join's output frontier won't advance and the capture +/// operator will never emit `Event::Progress` messages. +pub struct SinkHandle { + /// Input for the WS thread to send client connect/disconnect events. + pub client_input: ClientInput, + /// Receiver for diagnostic updates produced by the cross-join. + /// + /// Each `Event::Messages(time, vec)` contains `((client_id, update), time, diff)` + /// triples. The WS thread routes updates to clients by `client_id`. + pub output_receiver: mpsc::Receiver, + /// The reference instant for computing elapsed durations. + /// Use `start.elapsed()` when calling `client_input.advance()`. + pub start: Instant, +} + +/// Everything returned by [`register`]. +pub struct LoggingState { + pub traces: LoggingTraces, + pub sink: SinkHandle, +} + +/// All trace handles. +pub struct LoggingTraces { + pub timely: TimelyTraces, + pub differential: DifferentialTraces, +} + +// ============================================================================ +// Timestamp quantization +// ============================================================================ + +/// Default quantization interval. +const INTERVAL: Duration = Duration::from_secs(1); + +/// Round a Duration up to the next multiple of `interval`. +fn quantize(time: Duration, interval: Duration) -> Duration { + let nanos = time.as_nanos(); + let interval_nanos = interval.as_nanos(); + let rounded = (nanos / interval_nanos + 1) * interval_nanos; + Duration::from_nanos(rounded as u64) +} + +/// Quantize timestamps in a collection's inner stream. +fn quantize_collection( + collection: VecCollection, + interval: Duration, +) -> VecCollection +where + S: Scope, + D: differential_dataflow::Data, +{ + collection + .inner + .map_in_place(move |(_, time, _)| *time = quantize(*time, interval)) + .as_collection() +} + +// ============================================================================ +// Registration +// ============================================================================ + +/// Register diagnostics logging for a worker. +/// +/// Builds a dataflow that: +/// 1. Captures timely and differential logging events into DD collections. +/// 2. Arranges them into indexed traces for persistence. +/// 3. Cross-joins all collections with a client input, so new clients +/// automatically receive the full current state as updates. +/// 4. Captures the output for the WebSocket thread via `mpsc`. +/// +/// If `log_logging` is true, the diagnostics dataflow itself will appear in +/// the timely logs. +/// +/// Returns a [`LoggingState`] with trace handles and a [`SinkHandle`] for +/// the WebSocket thread. +pub fn register(worker: &mut Worker, log_logging: bool) -> LoggingState { + let start = Instant::now(); + + // Event links for logging capture (worker-internal, Rc-based). + let t_link: Rc>> = Rc::new(EventLink::new()); + let d_link: Rc>> = + Rc::new(EventLink::new()); + + // Cross-thread channels for client input and diagnostic output. + let (client_tx, client_rx) = mpsc::channel::>(); + let (output_tx, output_rx) = mpsc::channel::(); + + if log_logging { + install_loggers(worker, t_link.clone(), d_link.clone()); + } + + let traces = worker.dataflow::(|scope| { + // Replay logging events into the dataflow. + let timely_stream = Some(t_link.clone()).replay_into(scope); + let diff_stream = Some(d_link.clone()).replay_into(scope); + + // Build collections and arrangements. + let (t_traces, t_collections) = construct_timely(scope, timely_stream); + let (d_traces, d_collections) = construct_differential(scope, diff_stream); + + // Replay client connection events from the WS thread. + let client_iter = MpscEventIterator { receiver: client_rx }; + let clients: VecCollection<_, usize, i64> = + Some(client_iter).replay_into(scope).as_collection(); + + // Cross-join: clients × each data collection. + let clients_keyed = clients.map(|c| ((), c)); + + // Tag all collections and cross-join with clients. + let operators_tagged = t_collections.operators + .map(|(id, name, addr)| ((), DiagnosticUpdate::Operator { id, name, addr })); + let channels_tagged = t_collections.channels + .map(|(id, scope_addr, source, target)| { + ((), DiagnosticUpdate::Channel { id, scope_addr, source, target }) + }); + + // Key-only stats: tag them all and concat. + let stats = concatenate(scope, vec![ + t_collections.elapsed + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::Elapsed, id })), + t_collections.messages + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::Messages, id })), + d_collections.arrangement_batches + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::ArrangementBatches, id })), + d_collections.arrangement_records + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::ArrangementRecords, id })), + d_collections.sharing + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::Sharing, id })), + d_collections.batcher_records + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::BatcherRecords, id })), + d_collections.batcher_size + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::BatcherSize, id })), + d_collections.batcher_capacity + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::BatcherCapacity, id })), + d_collections.batcher_allocations + .map(|id| ((), DiagnosticUpdate::Stat { kind: StatKind::BatcherAllocations, id })), + ]); + + // Concatenate all tagged collections. + let all_data = concatenate(scope, vec![ + operators_tagged, + channels_tagged, + stats, + ]); + + let output = clients_keyed + .join(all_data) + .map(|((), (client_id, update))| (client_id, update)); + + // Route all output to worker 0 before capture, since only worker 0 + // runs the WebSocket server. + output.inner.exchange(|_| 0).capture_into(output_tx); + + LoggingTraces { + timely: t_traces, + differential: d_traces, + } + }); + + if !log_logging { + install_loggers(worker, t_link, d_link); + } + + LoggingState { + traces, + sink: SinkHandle { + client_input: ClientInput { + sender: client_tx, + time: Duration::default(), + }, + output_receiver: output_rx, + start, + }, + } +} + +fn install_loggers( + worker: &mut Worker, + t_link: Rc>>, + d_link: Rc>>, +) { + let mut registry = worker.log_register().expect("Logging not initialized"); + + // Use timely's BatchLogger directly — it handles progress tracking + // with Duration timestamps, matching the logging framework's epoch. + let mut t_batch = BatchLogger::new(t_link); + registry.insert::("timely", move |time, data| { + t_batch.publish_batch(time, data); + }); + + let mut d_batch = BatchLogger::new(d_link); + registry.insert::("differential/arrange", move |time, data| { + d_batch.publish_batch(time, data); + }); +} + +// ============================================================================ +// Timely event demux +// ============================================================================ + +/// Internal: collections before arrangement, used for the cross-join. +struct TimelyCollections { + operators: VecCollection), i64>, + channels: VecCollection, (usize, usize), (usize, usize)), i64>, + elapsed: VecCollection, + messages: VecCollection, +} + +#[derive(Default)] +struct TimelyDemuxState { + operators: BTreeMap, + schedule_starts: BTreeMap, +} + +/// Build timely logging collections and arrangements. +fn construct_timely>( + scope: &mut S, + stream: Stream>, +) -> (TimelyTraces, TimelyCollections) { + type OpUpdate = ((usize, String, Vec), Duration, i64); + type ChUpdate = ((usize, Vec, (usize, usize), (usize, usize)), Duration, i64); + type ElUpdate = (usize, Duration, i64); + type MsgUpdate = (usize, Duration, i64); + + let mut demux = OperatorBuilder::new("Timely Demux".to_string(), scope.clone()); + let mut input = demux.new_input(stream, Pipeline); + + let (op_out, operates) = demux.new_output::>(); + let mut op_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(op_out); + let (ch_out, channels) = demux.new_output::>(); + let mut ch_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(ch_out); + let (el_out, elapsed) = demux.new_output::>(); + let mut el_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(el_out); + let (msg_out, messages) = demux.new_output::>(); + let mut msg_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(msg_out); + + demux.build(|_capabilities| { + let mut state = TimelyDemuxState::default(); + move |_frontiers| { + let mut op_act = op_out.activate(); + let mut ch_act = ch_out.activate(); + let mut el_act = el_out.activate(); + let mut msg_act = msg_out.activate(); + + input.for_each(|cap, data: &mut Vec<(Duration, TimelyEvent)>| { + let mut ops = op_act.session(&cap); + let mut chs = ch_act.session(&cap); + let mut els = el_act.session(&cap); + let mut msgs = msg_act.session(&cap); + let ts = *cap.time(); + + for (event_time, event) in data.drain(..) { + match event { + TimelyEvent::Operates(e) => { + ops.give(((e.id, e.name.clone(), e.addr.clone()), ts, 1i64)); + state.operators.insert(e.id, e); + } + TimelyEvent::Shutdown(e) => { + if let Some(op) = state.operators.remove(&e.id) { + ops.give(((op.id, op.name, op.addr), ts, -1i64)); + } + } + TimelyEvent::Channels(e) => { + chs.give(( + (e.id, e.scope_addr.clone(), e.source, e.target), + ts, + 1i64, + )); + } + TimelyEvent::Schedule(e) => match e.start_stop { + StartStop::Start => { + state.schedule_starts.insert(e.id, event_time); + } + StartStop::Stop => { + if let Some(start) = state.schedule_starts.remove(&e.id) { + let elapsed_ns = + event_time.saturating_sub(start).as_nanos() as i64; + if elapsed_ns > 0 { + els.give((e.id, ts, elapsed_ns)); + } + } + } + }, + TimelyEvent::Messages(e) => { + if e.is_send { + msgs.give((e.channel, ts, e.record_count as i64)); + } + } + _ => {} + } + } + }); + } + }); + + // Quantize timestamps to interval boundaries. + let op_collection = quantize_collection(operates.as_collection(), INTERVAL); + let ch_collection = quantize_collection(channels.as_collection(), INTERVAL); + let el_collection = quantize_collection(elapsed.as_collection(), INTERVAL); + let msg_collection = quantize_collection(messages.as_collection(), INTERVAL); + + // Arrange for persistence. + let operators = op_collection.clone() + .map(|(id, name, addr)| (id, (name, addr))) + .arrange_by_key_named("Arrange Operators"); + let channels = ch_collection.clone() + .map(|(id, scope_addr, source, target)| (id, (scope_addr, source, target))) + .arrange_by_key_named("Arrange Channels"); + let elapsed = el_collection.clone() + .arrange_by_self_named("Arrange Elapsed"); + let messages = msg_collection.clone() + .arrange_by_self_named("Arrange Messages"); + + let traces = TimelyTraces { + operators: operators.trace, + channels: channels.trace, + elapsed: elapsed.trace, + messages: messages.trace, + }; + + let collections = TimelyCollections { + operators: op_collection, + channels: ch_collection, + elapsed: el_collection, + messages: msg_collection, + }; + + (traces, collections) +} + +// ============================================================================ +// Differential event demux +// ============================================================================ + +/// Internal: collections before arrangement, used for the cross-join. +struct DifferentialCollections { + arrangement_batches: VecCollection, + arrangement_records: VecCollection, + sharing: VecCollection, + batcher_records: VecCollection, + batcher_size: VecCollection, + batcher_capacity: VecCollection, + batcher_allocations: VecCollection, +} + +/// Build differential logging collections and arrangements. +fn construct_differential>( + scope: &mut S, + stream: Stream>, +) -> (DifferentialTraces, DifferentialCollections) { + type Update = (usize, Duration, i64); + + let mut demux = OperatorBuilder::new("Differential Demux".to_string(), scope.clone()); + let mut input = demux.new_input(stream, Pipeline); + + let (bat_out, batches) = demux.new_output::>(); + let mut bat_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(bat_out); + let (rec_out, records) = demux.new_output::>(); + let mut rec_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(rec_out); + let (shr_out, sharing) = demux.new_output::>(); + let mut shr_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(shr_out); + let (br_out, batcher_records) = demux.new_output::>(); + let mut br_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(br_out); + let (bs_out, batcher_size) = demux.new_output::>(); + let mut bs_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(bs_out); + let (bc_out, batcher_capacity) = demux.new_output::>(); + let mut bc_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(bc_out); + let (ba_out, batcher_allocations) = demux.new_output::>(); + let mut ba_out = OutputBuilder::<_, CapacityContainerBuilder>>::from(ba_out); + + demux.build(|_capabilities| { + move |_frontiers| { + let mut bat_act = bat_out.activate(); + let mut rec_act = rec_out.activate(); + let mut shr_act = shr_out.activate(); + let mut br_act = br_out.activate(); + let mut bs_act = bs_out.activate(); + let mut bc_act = bc_out.activate(); + let mut ba_act = ba_out.activate(); + + input.for_each(|cap, data: &mut Vec<(Duration, DifferentialEvent)>| { + let mut bat = bat_act.session(&cap); + let mut rec = rec_act.session(&cap); + let mut shr = shr_act.session(&cap); + let mut b_rec = br_act.session(&cap); + let mut b_sz = bs_act.session(&cap); + let mut b_cap = bc_act.session(&cap); + let mut b_alloc = ba_act.session(&cap); + let ts = *cap.time(); + + for (_event_time, event) in data.drain(..) { + match event { + DifferentialEvent::Batch(e) => { + bat.give((e.operator, ts, 1i64)); + rec.give((e.operator, ts, e.length as i64)); + } + DifferentialEvent::Merge(e) => { + if let Some(complete) = e.complete { + bat.give((e.operator, ts, -1i64)); + let diff = complete as i64 - (e.length1 + e.length2) as i64; + if diff != 0 { + rec.give((e.operator, ts, diff)); + } + } + } + DifferentialEvent::Drop(e) => { + bat.give((e.operator, ts, -1i64)); + let diff = -(e.length as i64); + if diff != 0 { + rec.give((e.operator, ts, diff)); + } + } + DifferentialEvent::TraceShare(e) => { + shr.give((e.operator, ts, e.diff as i64)); + } + DifferentialEvent::Batcher(e) => { + b_rec.give((e.operator, ts, e.records_diff as i64)); + b_sz.give((e.operator, ts, e.size_diff as i64)); + b_cap.give((e.operator, ts, e.capacity_diff as i64)); + b_alloc.give((e.operator, ts, e.allocations_diff as i64)); + } + _ => {} + } + } + }); + } + }); + + // Quantize timestamps to interval boundaries. + let bat_coll = quantize_collection(batches.as_collection(), INTERVAL); + let rec_coll = quantize_collection(records.as_collection(), INTERVAL); + let shr_coll = quantize_collection(sharing.as_collection(), INTERVAL); + let br_coll = quantize_collection(batcher_records.as_collection(), INTERVAL); + let bs_coll = quantize_collection(batcher_size.as_collection(), INTERVAL); + let bc_coll = quantize_collection(batcher_capacity.as_collection(), INTERVAL); + let ba_coll = quantize_collection(batcher_allocations.as_collection(), INTERVAL); + + let traces = DifferentialTraces { + arrangement_batches: bat_coll.clone().arrange_by_self_named("Arrange ArrangementBatches").trace, + arrangement_records: rec_coll.clone().arrange_by_self_named("Arrange ArrangementRecords").trace, + sharing: shr_coll.clone().arrange_by_self_named("Arrange Sharing").trace, + batcher_records: br_coll.clone().arrange_by_self_named("Arrange BatcherRecords").trace, + batcher_size: bs_coll.clone().arrange_by_self_named("Arrange BatcherSize").trace, + batcher_capacity: bc_coll.clone().arrange_by_self_named("Arrange BatcherCapacity").trace, + batcher_allocations: ba_coll.clone().arrange_by_self_named("Arrange BatcherAllocations").trace, + }; + + let collections = DifferentialCollections { + arrangement_batches: bat_coll, + arrangement_records: rec_coll, + sharing: shr_coll, + batcher_records: br_coll, + batcher_size: bs_coll, + batcher_capacity: bc_coll, + batcher_allocations: ba_coll, + }; + + (traces, collections) +} diff --git a/diagnostics/src/server.rs b/diagnostics/src/server.rs new file mode 100644 index 000000000..a0a40c2b6 --- /dev/null +++ b/diagnostics/src/server.rs @@ -0,0 +1,242 @@ +//! WebSocket server that bridges the diagnostics dataflow to browser clients. +//! +//! The server runs on a background thread and manages the full client lifecycle: +//! +//! 1. Accepts WebSocket connections and assigns each a unique client ID. +//! 2. Announces connects/disconnects to the diagnostics dataflow via the +//! [`SinkHandle`](crate::logging::SinkHandle)'s client input channel. +//! 3. Reads diagnostic updates from the capture channel (produced by the +//! dataflow's cross-join of clients × data) and forwards them as JSON +//! to the appropriate browser client. +//! +//! This server only handles the WebSocket data protocol. The browser loads +//! `index.html` (and its JavaScript) from a separate static file server +//! (e.g., `python3 -m http.server 8000`). A future improvement could embed +//! static file serving here so only one port is needed. + +use std::collections::HashMap; +use std::net::TcpListener; +use std::sync::mpsc; +use std::thread; +use std::time::Duration; + +use serde_json; +use tungstenite::{Message, accept}; + +use timely::dataflow::operators::capture::Event; + +use crate::logging::{DiagnosticUpdate, SinkHandle, StatKind}; + +/// A running diagnostics WebSocket server. +/// +/// Created by [`Server::start`]. The server thread runs in the background +/// until the `Server` is dropped. +pub struct Server { + _handle: thread::JoinHandle<()>, +} + +impl Server { + /// Start the diagnostics WebSocket server on the given port. + /// + /// Takes ownership of the [`SinkHandle`] and moves it to a background + /// thread. When a browser connects to `ws://localhost:{port}`: + /// + /// 1. The server announces the new client to the diagnostics dataflow. + /// 2. The dataflow's cross-join produces the full current state as diffs. + /// 3. The server serializes those diffs as JSON and sends them over the + /// WebSocket, followed by incremental updates as the computation runs. + /// + /// The browser should load `index.html` over HTTP (not `file://`), since + /// browsers restrict WebSocket connections from `file://` origins. Serve + /// the diagnostics directory with any static file server, e.g.: + /// + /// ```text + /// cd diagnostics && python3 -m http.server 8000 + /// ``` + /// + /// Then open `http://localhost:8000/index.html` and click Connect. + /// + /// # Panics + /// + /// Panics if the port cannot be bound. + pub fn start(port: u16, sink: SinkHandle) -> Self { + let handle = thread::spawn(move || run_server(port, sink)); + eprintln!("Diagnostics server on ws://localhost:{port}"); + Server { _handle: handle } + } +} + +/// JSON-serializable update sent to clients. +#[derive(serde::Serialize)] +#[serde(tag = "type")] +enum JsonUpdate<'a> { + Operator { + id: usize, + name: &'a str, + addr: &'a [usize], + diff: i64, + }, + Channel { + id: usize, + scope_addr: &'a [usize], + source: (usize, usize), + target: (usize, usize), + diff: i64, + }, + Stat { + kind: &'a str, + id: usize, + diff: i64, + }, +} + +fn stat_kind_str(kind: &StatKind) -> &'static str { + match kind { + StatKind::Elapsed => "Elapsed", + StatKind::Messages => "Messages", + StatKind::ArrangementBatches => "ArrangementBatches", + StatKind::ArrangementRecords => "ArrangementRecords", + StatKind::Sharing => "Sharing", + StatKind::BatcherRecords => "BatcherRecords", + StatKind::BatcherSize => "BatcherSize", + StatKind::BatcherCapacity => "BatcherCapacity", + StatKind::BatcherAllocations => "BatcherAllocations", + } +} + +fn update_to_json(update: &DiagnosticUpdate, diff: i64) -> serde_json::Value { + match update { + DiagnosticUpdate::Operator { id, name, addr } => { + serde_json::to_value(JsonUpdate::Operator { + id: *id, + name, + addr, + diff, + }) + .unwrap() + } + DiagnosticUpdate::Channel { + id, + scope_addr, + source, + target, + } => serde_json::to_value(JsonUpdate::Channel { + id: *id, + scope_addr, + source: *source, + target: *target, + diff, + }) + .unwrap(), + DiagnosticUpdate::Stat { kind, id } => { + serde_json::to_value(JsonUpdate::Stat { + kind: stat_kind_str(kind), + id: *id, + diff, + }) + .unwrap() + } + } +} + +const FLUSH_INTERVAL: Duration = Duration::from_millis(100); + +fn run_server(port: u16, sink: SinkHandle) { + 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 client_input = sink.client_input; + let receiver = sink.output_receiver; + let start = sink.start; + + let mut clients: HashMap> = HashMap::new(); + let mut next_client_id: usize = 0; + + loop { + // Accept pending connections. + loop { + match listener.accept() { + Ok((stream, addr)) => { + eprintln!("Diagnostics client connected from {addr}"); + stream.set_nonblocking(false).ok(); + match accept(stream) { + Ok(ws) => { + let client_id = next_client_id; + next_client_id += 1; + clients.insert(client_id, ws); + + // Announce to the diagnostics dataflow. + client_input.connect(client_id, start.elapsed()); + eprintln!(" assigned client id {client_id}"); + } + 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 diagnostic updates and group by client. + let mut batches_by_client: HashMap> = HashMap::new(); + loop { + match receiver.try_recv() { + Ok(Event::Messages(_time, data)) => { + for ((client_id, update), _ts, diff) in data { + let json = update_to_json(&update, diff); + batches_by_client.entry(client_id).or_default().push(json); + } + } + Ok(Event::Progress(_)) => {} + Err(mpsc::TryRecvError::Empty) => break, + Err(mpsc::TryRecvError::Disconnected) => { + eprintln!("Diagnostics output channel closed, shutting down server"); + // Close all clients gracefully. + for (_, mut ws) in clients.drain() { + let _ = ws.close(None); + } + return; + } + } + } + + // Send batched updates to each client. + let mut disconnected = Vec::new(); + for (client_id, updates) in &batches_by_client { + if let Some(ws) = clients.get_mut(client_id) { + if !updates.is_empty() { + let payload = serde_json::to_string(updates).unwrap(); + if ws.send(Message::Text(payload.into())).is_err() { + disconnected.push(*client_id); + } + } + } + } + + // Handle disconnects (also check for clients that closed their end). + for (client_id, ws) in clients.iter_mut() { + // Non-blocking read to detect closed connections. + // tungstenite in blocking mode would block here, so we just + // check on send failure above. + let _ = ws; // placeholder — send failure detection above is sufficient + let _ = client_id; + } + for client_id in disconnected { + clients.remove(&client_id); + client_input.disconnect(client_id, start.elapsed()); + eprintln!("Diagnostics client {client_id} disconnected"); + } + + // Advance time periodically even without client events, so the + // dataflow frontier can progress. + client_input.advance(start.elapsed()); + + std::thread::sleep(FLUSH_INTERVAL); + } +} diff --git a/differential-dataflow/examples/columnar.rs b/differential-dataflow/examples/columnar.rs deleted file mode 100644 index 9a97922f1..000000000 --- a/differential-dataflow/examples/columnar.rs +++ /dev/null @@ -1,1304 +0,0 @@ -//! Wordcount based on `columnar`. - -use timely::container::{ContainerBuilder, PushInto}; -use timely::dataflow::InputHandle; -use timely::dataflow::ProbeHandle; - -use differential_dataflow::operators::arrange::arrangement::arrange_core; - -use mimalloc::MiMalloc; - -#[global_allocator] -static GLOBAL: MiMalloc = MiMalloc; - -fn main() { - - type WordCount = (Vec, u64, i64); - type Builder = KeyColBuilder; - - let keys: usize = std::env::args().nth(1).expect("missing argument 1").parse().unwrap(); - let size: usize = std::env::args().nth(2).expect("missing argument 2").parse().unwrap(); - - let timer1 = ::std::time::Instant::now(); - let timer2 = timer1.clone(); - - // initializes and runs a timely dataflow. - timely::execute_from_args(std::env::args(), move |worker| { - - let mut data_input = >::new_with_builder(); - let mut keys_input = >::new_with_builder(); - let mut probe = ProbeHandle::new(); - - // create a new input, exchange data, and inspect its output - worker.dataflow::(|scope| { - - let data = data_input.to_stream(scope); - let keys = keys_input.to_stream(scope); - - use differential_dataflow::Hashable; - let data_pact = KeyPact { hashfunc: |k: columnar::Ref<'_, Vec>| k.hashed() }; - let keys_pact = KeyPact { hashfunc: |k: columnar::Ref<'_, Vec>| k.hashed() }; - - let data = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(data, data_pact, "Data"); - let keys = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(keys, keys_pact, "Keys"); - - keys.join_core(data, |_k, (), ()| { Option::<()>::None }) - .probe_with(&mut probe); - }); - - // Resources for placing input data in containers. - use std::fmt::Write; - let mut buffer = String::default(); - let mut builder = KeyColBuilder::::default(); - - // Load up data in batches. - let mut counter = 0; - while counter < 10 * keys { - let mut i = worker.index(); - let time = *data_input.time(); - while i < size { - let val = (counter + i) % keys; - write!(buffer, "{:?}", val).unwrap(); - builder.push_into((buffer.as_bytes(), time, 1)); - buffer.clear(); - i += worker.peers(); - } - while let Some(container) = builder.finish() { - data_input.send_batch(container); - } - counter += size; - data_input.advance_to(data_input.time() + 1); - keys_input.advance_to(keys_input.time() + 1); - while probe.less_than(data_input.time()) { - worker.step_or_park(None); - } - } - println!("{:?}\tloading complete", timer1.elapsed()); - - let mut queries = 0; - while queries < 10 * keys { - let mut i = worker.index(); - let time = *data_input.time(); - while i < size { - let val = (queries + i) % keys; - write!(buffer, "{:?}", val).unwrap(); - builder.push_into((buffer.as_bytes(), time, 1)); - buffer.clear(); - i += worker.peers(); - } - while let Some(container) = builder.finish() { - keys_input.send_batch(container); - } - queries += size; - data_input.advance_to(data_input.time() + 1); - keys_input.advance_to(keys_input.time() + 1); - while probe.less_than(data_input.time()) { - worker.step_or_park(None); - } - } - println!("{:?}\tqueries complete", timer1.elapsed()); - - }) - .unwrap(); - - println!("{:?}\tshut down", timer2.elapsed()); -} - -pub use layout::{ColumnarLayout, ColumnarUpdate}; -pub mod layout { - - use std::fmt::Debug; - use columnar::Columnar; - use differential_dataflow::trace::implementations::{Layout, OffsetList}; - use differential_dataflow::difference::Semigroup; - use differential_dataflow::lattice::Lattice; - use timely::progress::Timestamp; - - /// A layout based on columnar - pub struct ColumnarLayout { - phantom: std::marker::PhantomData, - } - - impl ColumnarUpdate for (K, V, T, R) - where - K: Columnar + Debug + Ord + Clone + 'static, - V: Columnar + Debug + Ord + Clone + 'static, - T: Columnar + Debug + Ord + Default + Clone + Lattice + Timestamp, - R: Columnar + Debug + Ord + Default + Semigroup + 'static, - { - type Key = K; - type Val = V; - type Time = T; - type Diff = R; - } - - impl ColumnarUpdate for (K, T, R) - where - K: Columnar + Debug + Ord + Clone + 'static, - T: Columnar + Debug + Ord + Default + Clone + Lattice + Timestamp, - R: Columnar + Debug + Ord + Default + Semigroup + 'static, - { - type Key = K; - type Val = (); - type Time = T; - type Diff = R; - } - - - use crate::arrangement::Coltainer; - impl Layout for ColumnarLayout { - type KeyContainer = Coltainer; - type ValContainer = Coltainer; - type TimeContainer = Coltainer; - type DiffContainer = Coltainer; - type OffsetContainer = OffsetList; - } - - /// A type that names constituent update types. - /// - /// We will use their associated `Columnar::Container` - pub trait ColumnarUpdate : Debug + 'static { - type Key: Columnar + Debug + Ord + Clone + 'static; - type Val: Columnar + Debug + Ord + Clone + 'static; - type Time: Columnar + Debug + Ord + Default + Clone + Lattice + Timestamp; - type Diff: Columnar + Debug + Ord + Default + Semigroup + 'static; - } - - /// A container whose references can be ordered. - pub trait OrdContainer : for<'a> columnar::Container : Ord> { } - impl columnar::Container : Ord>> OrdContainer for C { } - -} - -pub use container::Column; -mod container { - - /// A container based on a columnar store, encoded in aligned bytes. - pub enum Column { - /// The typed variant of the container. - Typed(C), - /// The binary variant of the container. - Bytes(timely::bytes::arc::Bytes), - /// Relocated, aligned binary data, if `Bytes` doesn't work for some reason. - /// - /// Reasons could include misalignment, cloning of data, or wanting - /// to release the `Bytes` as a scarce resource. - Align(std::sync::Arc<[u64]>), - } - - impl Default for Column { - fn default() -> Self { Self::Typed(Default::default()) } - } - - impl Column { - pub fn as_mut(&mut self) -> &mut C { if let Column::Typed(c) = self { c } else { panic!() }} - } - - // The clone implementation moves out of the `Bytes` variant into `Align`. - // This is optional and non-optimal, as the bytes clone is relatively free. - // But, we don't want to leak the uses of `Bytes`, is why we do this I think. - impl Clone for Column where C: Clone { - fn clone(&self) -> Self { - match self { - Column::Typed(t) => Column::Typed(t.clone()), - Column::Bytes(b) => { - assert!(b.len() % 8 == 0); - let mut alloc: Vec = vec![0; b.len() / 8]; - bytemuck::cast_slice_mut(&mut alloc[..]).copy_from_slice(&b[..]); - Self::Align(alloc.into()) - }, - Column::Align(a) => Column::Align(std::sync::Arc::clone(&a.clone())), - } - } - fn clone_from(&mut self, other: &Self) { - match (self, other) { - (Column::Typed(t0), Column::Typed(t1)) => { - // Derived `Clone` implementations for e.g. tuples cannot be relied on to call `clone_from`. - let t1 = t1.borrow(); - t0.clear(); - t0.extend_from_self(t1, 0..t1.len()); - } - (Column::Align(a0), Column::Align(a1)) => { a0.clone_from(a1); } - (x, y) => { *x = y.clone(); } - } - } - } - - use columnar::{Len, FromBytes}; - use columnar::bytes::{EncodeDecode, Indexed}; - - impl Column { - /// Borrows the contents no matter their representation. - /// - /// This function is meant to be efficient, but it cannot be relied on to be zero-cost. - /// Ideal uses would borrow a container infrequently, and access the borrowed form repeatedly. - #[inline(always)] pub fn borrow(&self) -> C::Borrowed<'_> { - match self { - Column::Typed(t) => t.borrow(), - Column::Bytes(b) => as FromBytes>::from_bytes(&mut Indexed::decode(bytemuck::cast_slice(b))), - Column::Align(a) => as FromBytes>::from_bytes(&mut Indexed::decode(a)), - } - } - - pub fn into_typed(self) -> C where C: Default { - if let Column::Typed(c) = self { c } - else { - let mut result = C::default(); - let borrow = self.borrow(); - result.extend_from_self(borrow, 0 .. borrow.len()); - result - } - } - } - - impl timely::container::PushInto for Column where C: columnar::Push { - #[inline] - fn push_into(&mut self, item: T) { - match self { - Column::Typed(t) => t.push(item), - Column::Align(_) | Column::Bytes(_) => { - // We really oughtn't be calling this in this case. - // We could convert to owned, but need more constraints on `C`. - unimplemented!("Pushing into Column::Bytes without first clearing"); - } - } - } - } - - impl timely::dataflow::channels::ContainerBytes for Column { - fn from_bytes(bytes: timely::bytes::arc::Bytes) -> Self { - // Our expectation / hope is that `bytes` is `u64` aligned and sized. - // If the alignment is borked, we can relocate. IF the size is borked, - // not sure what we do in that case. - assert!(bytes.len() % 8 == 0); - if bytemuck::try_cast_slice::<_, u64>(&bytes).is_ok() { - Self::Bytes(bytes) - } - else { - println!("Re-locating bytes for alignment reasons"); - let mut alloc: Vec = vec![0; bytes.len() / 8]; - bytemuck::cast_slice_mut(&mut alloc[..]).copy_from_slice(&bytes[..]); - Self::Align(alloc.into()) - } - } - - // Borrow rather than trust the sizes of the bytes themselves. - fn length_in_bytes(&self) -> usize { 8 * Indexed::length_in_words(&self.borrow()) } - - // Borrow rather than trust the sizes of the bytes themselves. - fn into_bytes(&self, writer: &mut W) { Indexed::write(writer, &self.borrow()).unwrap() } - } -} - - -pub use storage::val::ValStorage; -pub use storage::key::KeyStorage; -pub mod storage { - - pub mod val { - - use std::fmt::Debug; - use columnar::{Borrow, Container, ContainerOf, Index, Len, Push}; - use columnar::Vecs; - - use crate::layout::ColumnarUpdate as Update; - - /// Trie-shaped update storage. - #[derive(Debug)] - pub struct ValStorage { - /// An ordered list of keys. - pub keys: ContainerOf, - /// For each key in `keys`, a list of values. - pub vals: Vecs>, - /// For each val in `vals`, a list of (time, diff) updates. - pub upds: Vecs<(ContainerOf, ContainerOf)>, - } - - impl Default for ValStorage { fn default() -> Self { Self { keys: Default::default(), vals: Default::default(), upds: Default::default(), } } } - impl Clone for ValStorage { fn clone(&self) -> Self { Self { keys: self.keys.clone(), vals: self.vals.clone(), upds: self.upds.clone(), } } } - - pub type Tuple = (::Key, ::Val, ::Time, ::Diff); - - use std::ops::Range; - impl ValStorage { - - /// Forms `Self` from sorted update tuples. - pub fn form<'a>(mut sorted: impl Iterator>>) -> Self { - - let mut output = Self::default(); - - if let Some((key,val,time,diff)) = sorted.next() { - output.keys.push(key); - output.vals.values.push(val); - output.upds.values.push((time, diff)); - for (key,val,time,diff) in sorted { - let mut differs = false; - // We would now iterate over layers. - // We'll do that manually, as the types are all different. - // Keys first; non-standard logic because they are not (yet) a list of lists. - let keys_len = output.keys.len(); - differs |= ContainerOf::::reborrow_ref(key) != output.keys.borrow().get(keys_len-1); - if differs { output.keys.push(key); } - // Vals next - let vals_len = output.vals.values.len(); - if differs { output.vals.bounds.push(vals_len as u64); } - differs |= ContainerOf::::reborrow_ref(val) != output.vals.values.borrow().get(vals_len-1); - if differs { output.vals.values.push(val); } - // Upds last - let upds_len = output.upds.values.len(); - if differs { output.upds.bounds.push(upds_len as u64); } - // differs |= ContainerOf::<(U::Time,U::Diff)>::reborrow_ref((time,diff)) != output.upds.values.borrow().get(upds_len-1); - differs = true; - if differs { output.upds.values.push((time,diff)); } - } - // output.keys.bounds.push(vals_len as u64); - output.vals.bounds.push(output.vals.values.len() as u64); - output.upds.bounds.push(output.upds.values.len() as u64); - } - - assert_eq!(output.keys.len(), output.vals.len()); - assert_eq!(output.vals.values.len(), output.upds.len()); - - output - } - - pub fn vals_bounds(&self, range: Range) -> Range { - if !range.is_empty() { - let lower = if range.start == 0 { 0 } else { Index::get(self.vals.bounds.borrow(), range.start-1) as usize }; - let upper = Index::get(self.vals.bounds.borrow(), range.end-1) as usize; - lower .. upper - } else { range } - } - - pub fn upds_bounds(&self, range: Range) -> Range { - if !range.is_empty() { - let lower = if range.start == 0 { 0 } else { Index::get(self.upds.bounds.borrow(), range.start-1) as usize }; - let upper = Index::get(self.upds.bounds.borrow(), range.end-1) as usize; - lower .. upper - } else { range } - } - - /// Copies `other[range]` into self, keys and all. - pub fn extend_from_keys(&mut self, other: &Self, range: Range) { - self.keys.extend_from_self(other.keys.borrow(), range.clone()); - self.vals.extend_from_self(other.vals.borrow(), range.clone()); - self.upds.extend_from_self(other.upds.borrow(), other.vals_bounds(range)); - } - - pub fn extend_from_vals(&mut self, other: &Self, range: Range) { - self.vals.values.extend_from_self(other.vals.values.borrow(), range.clone()); - self.upds.extend_from_self(other.upds.borrow(), range); - } - } - - impl timely::Accountable for ValStorage { - #[inline] fn record_count(&self) -> i64 { use columnar::Len; self.upds.values.len() as i64 } - } - - use timely::dataflow::channels::ContainerBytes; - impl ContainerBytes for ValStorage { - fn from_bytes(_bytes: timely::bytes::arc::Bytes) -> Self { unimplemented!() } - fn length_in_bytes(&self) -> usize { unimplemented!() } - fn into_bytes(&self, _writer: &mut W) { unimplemented!() } - } - } - - pub mod key { - - use columnar::{Borrow, Container, ContainerOf, Index, Len, Push}; - use columnar::Vecs; - - use crate::layout::ColumnarUpdate as Update; - use crate::Column; - - /// Trie-shaped update storage. - pub struct KeyStorage { - /// An ordered list of keys. - pub keys: Column>, - /// For each key in `keys`, a list of (time, diff) updates. - pub upds: Column, ContainerOf)>>, - } - - impl Default for KeyStorage { fn default() -> Self { Self { keys: Default::default(), upds: Default::default(), } } } - impl Clone for KeyStorage { fn clone(&self) -> Self { Self { keys: self.keys.clone(), upds: self.upds.clone(), } } } - - pub type Tuple = (::Key, ::Time, ::Diff); - - use std::ops::Range; - impl KeyStorage { - - /// Forms `Self` from sorted update tuples. - pub fn form<'a>(mut sorted: impl Iterator>>) -> Self { - - let mut keys: ContainerOf = Default::default(); - let mut upds: Vecs<(ContainerOf, ContainerOf)> = Default::default(); - - if let Some((key,time,diff)) = sorted.next() { - keys.push(key); - upds.values.push((time, diff)); - for (key,time,diff) in sorted { - let mut differs = false; - // We would now iterate over layers. - // We'll do that manually, as the types are all different. - // Keys first; non-standard logic because they are not (yet) a list of lists. - let keys_len = keys.borrow().len(); - differs |= ContainerOf::::reborrow_ref(key) != keys.borrow().get(keys_len-1); - if differs { keys.push(key); } - // Upds last - let upds_len = upds.borrow().values.len(); - if differs { upds.bounds.push(upds_len as u64); } - // differs |= ContainerOf::<(U::Time,U::Diff)>::reborrow_ref((time,diff)) != output.upds.values.borrow().get(upds_len-1); - differs = true; - if differs { upds.values.push((time,diff)); } - } - upds.bounds.push(upds.borrow().values.len() as u64); - } - - assert_eq!(keys.borrow().len(), upds.borrow().len()); - - Self { - keys: Column::Typed(keys), - upds: Column::Typed(upds), - } - } - - pub fn upds_bounds(&self, range: Range) -> Range { - if !range.is_empty() { - let lower = if range.start == 0 { 0 } else { Index::get(self.upds.borrow().bounds, range.start-1) as usize }; - let upper = Index::get(self.upds.borrow().bounds, range.end-1) as usize; - lower .. upper - } else { range } - } - - /// Copies `other[range]` into self, keys and all. - pub fn extend_from_keys(&mut self, other: &Self, range: Range) { - self.keys.as_mut().extend_from_self(other.keys.borrow(), range.clone()); - self.upds.as_mut().extend_from_self(other.upds.borrow(), range.clone()); - } - } - - impl timely::Accountable for KeyStorage { - #[inline] fn record_count(&self) -> i64 { use columnar::Len; self.upds.borrow().values.len() as i64 } - } - - use timely::dataflow::channels::ContainerBytes; - impl ContainerBytes for KeyStorage { - fn from_bytes(mut bytes: timely::bytes::arc::Bytes) -> Self { - let keys: Column> = ContainerBytes::from_bytes(bytes.clone()); - let _ = bytes.extract_to(keys.length_in_bytes()); - let upds = ContainerBytes::from_bytes(bytes); - Self { keys, upds } - } - fn length_in_bytes(&self) -> usize { self.keys.length_in_bytes() + self.upds.length_in_bytes() } - fn into_bytes(&self, writer: &mut W) { - self.keys.into_bytes(writer); - self.upds.into_bytes(writer); - } - } - } -} - -pub use column_builder::{val::ValBuilder as ValColBuilder, key::KeyBuilder as KeyColBuilder}; -mod column_builder { - - pub mod val { - - use std::collections::VecDeque; - use columnar::{Columnar, Clear, Len, Push}; - - use crate::layout::ColumnarUpdate as Update; - use crate::ValStorage; - - type TupleContainer = <(::Key, ::Val, ::Time, ::Diff) as Columnar>::Container; - - /// A container builder for `Column`. - pub struct ValBuilder { - /// Container that we're writing to. - current: TupleContainer, - /// Empty allocation. - empty: Option>, - /// Completed containers pending to be sent. - pending: VecDeque>, - } - - use timely::container::PushInto; - impl PushInto for ValBuilder where TupleContainer : Push { - #[inline] - fn push_into(&mut self, item: T) { - self.current.push(item); - if self.current.len() > 1024 * 1024 { - // TODO: Consolidate the batch? - use columnar::{Borrow, Index}; - let mut refs = self.current.borrow().into_index_iter().collect::>(); - refs.sort(); - let storage = ValStorage::form(refs.into_iter()); - self.pending.push_back(storage); - self.current.clear(); - } - } - } - - impl Default for ValBuilder { - fn default() -> Self { - ValBuilder { - current: Default::default(), - empty: None, - pending: Default::default(), - } - } - } - - use timely::container::{ContainerBuilder, LengthPreservingContainerBuilder}; - impl ContainerBuilder for ValBuilder { - type Container = ValStorage; - - #[inline] - fn extract(&mut self) -> Option<&mut Self::Container> { - if let Some(container) = self.pending.pop_front() { - self.empty = Some(container); - self.empty.as_mut() - } else { - None - } - } - - #[inline] - fn finish(&mut self) -> Option<&mut Self::Container> { - if !self.current.is_empty() { - // TODO: Consolidate the batch? - use columnar::{Borrow, Index}; - let mut refs = self.current.borrow().into_index_iter().collect::>(); - refs.sort(); - let storage = ValStorage::form(refs.into_iter()); - self.pending.push_back(storage); - self.current.clear(); - } - self.empty = self.pending.pop_front(); - self.empty.as_mut() - } - } - - impl LengthPreservingContainerBuilder for ValBuilder { } - } - - pub mod key { - - use std::collections::VecDeque; - use columnar::{Columnar, Clear, Len, Push}; - - use crate::layout::ColumnarUpdate as Update; - use crate::KeyStorage; - - type TupleContainer = <(::Key, ::Time, ::Diff) as Columnar>::Container; - - /// A container builder for `Column`. - pub struct KeyBuilder { - /// Container that we're writing to. - current: TupleContainer, - /// Empty allocation. - empty: Option>, - /// Completed containers pending to be sent. - pending: VecDeque>, - } - - use timely::container::PushInto; - impl PushInto for KeyBuilder where TupleContainer : Push { - #[inline] - fn push_into(&mut self, item: T) { - self.current.push(item); - if self.current.len() > 1024 * 1024 { - // TODO: Consolidate the batch? - use columnar::{Borrow, Index}; - let mut refs = self.current.borrow().into_index_iter().collect::>(); - refs.sort(); - let storage = KeyStorage::form(refs.into_iter()); - self.pending.push_back(storage); - self.current.clear(); - } - } - } - - impl Default for KeyBuilder { fn default() -> Self { KeyBuilder { current: Default::default(), empty: None, pending: Default::default(), } } } - - use timely::container::{ContainerBuilder, LengthPreservingContainerBuilder}; - impl ContainerBuilder for KeyBuilder { - type Container = KeyStorage; - - #[inline] - fn extract(&mut self) -> Option<&mut Self::Container> { - if let Some(container) = self.pending.pop_front() { - self.empty = Some(container); - self.empty.as_mut() - } else { - None - } - } - - #[inline] - fn finish(&mut self) -> Option<&mut Self::Container> { - if !self.current.is_empty() { - // TODO: Consolidate the batch? - use columnar::{Borrow, Index}; - let mut refs = self.current.borrow().into_index_iter().collect::>(); - refs.sort(); - let storage = KeyStorage::form(refs.into_iter()); - self.pending.push_back(storage); - self.current.clear(); - } - self.empty = self.pending.pop_front(); - self.empty.as_mut() - } - } - - impl LengthPreservingContainerBuilder for KeyBuilder { } - } -} - -use distributor::key::KeyPact; -mod distributor { - - pub mod key { - - use std::rc::Rc; - - use columnar::{Index, Len}; - use timely::logging::TimelyLogger; - use timely::dataflow::channels::pushers::{Exchange, exchange::Distributor}; - use timely::dataflow::channels::Message; - use timely::dataflow::channels::pact::{LogPuller, LogPusher, ParallelizationContract}; - use timely::progress::Timestamp; - use timely::worker::AsWorker; - - use crate::layout::ColumnarUpdate as Update; - use crate::KeyStorage; - - pub struct KeyDistributor { - marker: std::marker::PhantomData, - hashfunc: H, - } - - impl FnMut(columnar::Ref<'a, U::Key>)->u64> Distributor> for KeyDistributor { - fn partition>>>(&mut self, container: &mut KeyStorage, time: &T, pushers: &mut [P]) { - - use columnar::{ContainerOf, Vecs, Container, Push}; - use crate::Column; - - let in_keys = container.keys.borrow(); - let in_upds = container.upds.borrow(); - - // We build bespoke containers by determining the target for each key using `self.hashfunc`, and then copying in key and associated data. - // We bypass the container builders, which do much work to go from tuples to columnar containers, and we save time by avoiding that round trip. - let mut out_keys = vec![ContainerOf::::default(); pushers.len()]; - let mut out_upds = vec![Vecs::<(ContainerOf::, ContainerOf::)>::default(); pushers.len()]; - for index in 0 .. in_keys.len() { - let key = in_keys.get(index); - let idx = ((self.hashfunc)(key) as usize) % pushers.len(); - out_keys[idx].push(key); - out_upds[idx].extend_from_self(in_upds, index..index+1); - } - - for ((pusher, keys), upds) in pushers.iter_mut().zip(out_keys).zip(out_upds) { - let mut container = KeyStorage { keys: Column::Typed(keys), upds: Column::Typed(upds) }; - Message::push_at(&mut container, time.clone(), pusher); - } - } - fn flush>>>(&mut self, _time: &T, _pushers: &mut [P]) { } - fn relax(&mut self) { } - } - - pub struct KeyPact { pub hashfunc: H } - - // Exchange uses a `Box` because it cannot know what type of pushable will return from the allocator. - impl ParallelizationContract> for KeyPact - where - T: Timestamp, - U: Update, - H: for<'a> FnMut(columnar::Ref<'a, U::Key>)->u64 + 'static, - { - type Pusher = Exchange< - T, - LogPusher>>>>, - KeyDistributor - >; - type Puller = LogPuller>>>>; - - fn connect(self, allocator: &mut A, identifier: usize, address: Rc<[usize]>, logging: Option) -> (Self::Pusher, Self::Puller) { - let (senders, receiver) = allocator.allocate::>>(identifier, address); - let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, allocator.index(), i, identifier, logging.clone())).collect::>(); - let distributor = KeyDistributor { - marker: std::marker::PhantomData, - hashfunc: self.hashfunc, - }; - (Exchange::new(senders, distributor), LogPuller::new(receiver, allocator.index(), identifier, logging.clone())) - } - } - } -} - -pub use arrangement::{ValBatcher, ValBuilder, ValSpine, KeyBatcher, KeyBuilder, KeySpine}; -pub mod arrangement { - - use std::rc::Rc; - use differential_dataflow::trace::implementations::ord_neu::{OrdValBatch, OrdKeyBatch}; - use differential_dataflow::trace::rc_blanket_impls::RcBuilder; - use differential_dataflow::trace::implementations::spine_fueled::Spine; - - use crate::layout::ColumnarLayout; - - /// A trace implementation backed by columnar storage. - pub type ValSpine = Spine>>>; - /// A batcher for columnar storage. - pub type ValBatcher = ValBatcher2<(K,V,T,R)>; - /// A builder for columnar storage. - pub type ValBuilder = RcBuilder>; - - /// A trace implementation backed by columnar storage. - pub type KeySpine = Spine>>>; - /// A batcher for columnar storage - pub type KeyBatcher = KeyBatcher2<(K,T,R)>; - /// A builder for columnar storage - pub type KeyBuilder = RcBuilder>; - - /// A batch container implementation for Column. - pub use batch_container::Coltainer; - pub mod batch_container { - - use columnar::{Borrow, Columnar, Container, Clear, Push, Index, Len}; - use differential_dataflow::trace::implementations::BatchContainer; - - /// Container, anchored by `C` to provide an owned type. - pub struct Coltainer { - pub container: C::Container, - } - - impl Default for Coltainer { - fn default() -> Self { Self { container: Default::default() } } - } - - impl BatchContainer for Coltainer where for<'a> columnar::Ref<'a, C> : Ord { - - type ReadItem<'a> = columnar::Ref<'a, C>; - type Owned = C; - - #[inline(always)] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { C::into_owned(item) } - #[inline(always)] fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.copy_from(item) } - - #[inline(always)] fn push_ref(&mut self, item: Self::ReadItem<'_>) { self.container.push(item) } - #[inline(always)] fn push_own(&mut self, item: &Self::Owned) { self.container.push(item) } - - /// Clears the container. May not release resources. - fn clear(&mut self) { self.container.clear() } - - /// Creates a new container with sufficient capacity. - fn with_capacity(_size: usize) -> Self { Self::default() } - /// Creates a new container with sufficient capacity. - fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { - Self { - container: ::Container::with_capacity_for([cont1.container.borrow(), cont2.container.borrow()].into_iter()), - } - } - - /// Converts a read item into one with a narrower lifetime. - #[inline(always)] fn reborrow<'b, 'a: 'b>(item: Self::ReadItem<'a>) -> Self::ReadItem<'b> { columnar::ContainerOf::::reborrow_ref(item) } - - /// Reference to the element at this position. - #[inline(always)] fn index(&self, index: usize) -> Self::ReadItem<'_> { self.container.borrow().get(index) } - - #[inline(always)] fn len(&self) -> usize { self.container.len() } - } - } - - use crate::{ColumnarUpdate, ValStorage, KeyStorage}; - use differential_dataflow::trace::implementations::chainless_batcher as chainless; - type ValBatcher2 = chainless::Batcher<::Time, ValStorage>; - type KeyBatcher2 = chainless::Batcher<::Time, KeyStorage>; - pub mod batcher { - - use std::ops::Range; - use columnar::{Borrow, Columnar, Container, Index, Len, Push}; - use differential_dataflow::trace::implementations::chainless_batcher as chainless; - use differential_dataflow::difference::{Semigroup, IsZero}; - use timely::progress::frontier::{Antichain, AntichainRef}; - - use crate::ColumnarUpdate as Update; - use crate::{ValStorage, KeyStorage}; - - impl chainless::BatcherStorage for ValStorage { - - fn len(&self) -> usize { self.upds.values.len() } - - #[inline(never)] - fn merge(self, other: Self) -> Self { - - let mut this_sum = U::Diff::default(); - let mut that_sum = U::Diff::default(); - - let mut merged = Self::default(); - let this = self; - let that = other; - let this_keys = this.keys.borrow(); - let that_keys = that.keys.borrow(); - let mut this_key_range = 0 .. this_keys.len(); - let mut that_key_range = 0 .. that_keys.len(); - while !this_key_range.is_empty() && !that_key_range.is_empty() { - let this_key = this_keys.get(this_key_range.start); - let that_key = that_keys.get(that_key_range.start); - match this_key.cmp(&that_key) { - std::cmp::Ordering::Less => { - let lower = this_key_range.start; - gallop(this_keys, &mut this_key_range, |x| x < that_key); - merged.extend_from_keys(&this, lower .. this_key_range.start); - }, - std::cmp::Ordering::Equal => { - // keys are equal; must make a bespoke vals list. - // only push the key if merged.vals.values.len() advances. - let values_len = merged.vals.values.len(); - let mut this_val_range = this.vals_bounds(this_key_range.start .. this_key_range.start+1); - let mut that_val_range = that.vals_bounds(that_key_range.start .. that_key_range.start+1); - while !this_val_range.is_empty() && !that_val_range.is_empty() { - let this_val = this.vals.values.borrow().get(this_val_range.start); - let that_val = that.vals.values.borrow().get(that_val_range.start); - match this_val.cmp(&that_val) { - std::cmp::Ordering::Less => { - let lower = this_val_range.start; - gallop(this.vals.values.borrow(), &mut this_val_range, |x| x < that_val); - merged.extend_from_vals(&this, lower .. this_val_range.start); - }, - std::cmp::Ordering::Equal => { - // vals are equal; must make a bespoke upds list. - // only push the val if merged.upds.values.len() advances. - let updates_len = merged.upds.values.len(); - let mut this_upd_range = this.upds_bounds(this_val_range.start .. this_val_range.start+1); - let mut that_upd_range = that.upds_bounds(that_val_range.start .. that_val_range.start+1); - - while !this_upd_range.is_empty() && !that_upd_range.is_empty() { - let (this_time, this_diff) = this.upds.values.borrow().get(this_upd_range.start); - let (that_time, that_diff) = that.upds.values.borrow().get(that_upd_range.start); - match this_time.cmp(&that_time) { - std::cmp::Ordering::Less => { - let lower = this_upd_range.start; - gallop(this.upds.values.0.borrow(), &mut this_upd_range, |x| x < that_time); - merged.upds.values.extend_from_self(this.upds.values.borrow(), lower .. this_upd_range.start); - }, - std::cmp::Ordering::Equal => { - // times are equal; must add diffs. - this_sum.copy_from(this_diff); - that_sum.copy_from(that_diff); - this_sum.plus_equals(&that_sum); - if !this_sum.is_zero() { merged.upds.values.push((this_time, &this_sum)); } - // Advance the update ranges by one. - this_upd_range.start += 1; - that_upd_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_upd_range.start; - gallop(that.upds.values.0.borrow(), &mut that_upd_range, |x| x < this_time); - merged.upds.values.extend_from_self(that.upds.values.borrow(), lower .. that_upd_range.start); - }, - } - } - // Extend with the remaining this and that updates. - merged.upds.values.extend_from_self(this.upds.values.borrow(), this_upd_range); - merged.upds.values.extend_from_self(that.upds.values.borrow(), that_upd_range); - // Seal the updates and push the val. - if merged.upds.values.len() > updates_len { - merged.upds.bounds.push(merged.upds.values.len() as u64); - merged.vals.values.push(this_val); - } - // Advance the val ranges by one. - this_val_range.start += 1; - that_val_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_val_range.start; - gallop(that.vals.values.borrow(), &mut that_val_range, |x| x < this_val); - merged.extend_from_vals(&that, lower .. that_val_range.start); - }, - } - } - // Extend with the remaining this and that values. - merged.extend_from_vals(&this, this_val_range); - merged.extend_from_vals(&that, that_val_range); - // Seal the values and push the key. - if merged.vals.values.len() > values_len { - merged.vals.bounds.push(merged.vals.values.len() as u64); - merged.keys.push(this_key); - } - // Advance the key ranges by one. - this_key_range.start += 1; - that_key_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_key_range.start; - gallop(that_keys, &mut that_key_range, |x| x < this_key); - merged.extend_from_keys(&that, lower .. that_key_range.start); - }, - } - } - // Extend with the remaining this and that keys. - merged.extend_from_keys(&this, this_key_range); - merged.extend_from_keys(&that, that_key_range); - - merged - } - - #[inline(never)] - fn split(&mut self, frontier: AntichainRef) -> Self { - // Unfortunately the times are at the leaves, so there can be no bulk copying. - let mut ship = Self::default(); - let mut keep = Self::default(); - let mut time = U::Time::default(); - for key_idx in 0 .. self.keys.len() { - let key = self.keys.borrow().get(key_idx); - let keep_vals_len = keep.vals.values.len(); - let ship_vals_len = ship.vals.values.len(); - for val_idx in self.vals_bounds(key_idx..key_idx+1) { - let val = self.vals.values.borrow().get(val_idx); - let keep_upds_len = keep.upds.values.len(); - let ship_upds_len = ship.upds.values.len(); - for upd_idx in self.upds_bounds(val_idx..val_idx+1) { - let (t, diff) = self.upds.values.borrow().get(upd_idx); - time.copy_from(t); - if frontier.less_equal(&time) { - keep.upds.values.push((t, diff)); - } - else { - ship.upds.values.push((t, diff)); - } - } - if keep.upds.values.len() > keep_upds_len { - keep.upds.bounds.push(keep.upds.values.len() as u64); - keep.vals.values.push(val); - } - if ship.upds.values.len() > ship_upds_len { - ship.upds.bounds.push(ship.upds.values.len() as u64); - ship.vals.values.push(val); - } - } - if keep.vals.values.len() > keep_vals_len { - keep.vals.bounds.push(keep.vals.values.len() as u64); - keep.keys.push(key); - } - if ship.vals.values.len() > ship_vals_len { - ship.vals.bounds.push(ship.vals.values.len() as u64); - ship.keys.push(key); - } - } - - *self = keep; - ship - } - - fn lower(&self, frontier: &mut Antichain) { - use columnar::Columnar; - let mut times = self.upds.values.0.borrow().into_index_iter(); - if let Some(time_ref) = times.next() { - let mut time = ::into_owned(time_ref); - frontier.insert_ref(&time); - for time_ref in times { - ::copy_from(&mut time, time_ref); - frontier.insert_ref(&time); - } - } - } - } - - impl chainless::BatcherStorage for KeyStorage { - - fn len(&self) -> usize { self.upds.borrow().values.len() } - - #[inline(never)] - fn merge(self, other: Self) -> Self { - - let mut this_sum = U::Diff::default(); - let mut that_sum = U::Diff::default(); - - let mut merged = Self::default(); - - let this = self; - let that = other; - let this_keys = this.keys.borrow(); - let that_keys = that.keys.borrow(); - let mut this_key_range = 0 .. this_keys.len(); - let mut that_key_range = 0 .. that_keys.len(); - let this_upds = this.upds.borrow(); - let that_upds = that.upds.borrow(); - - while !this_key_range.is_empty() && !that_key_range.is_empty() { - let this_key = this_keys.get(this_key_range.start); - let that_key = that_keys.get(that_key_range.start); - match this_key.cmp(&that_key) { - std::cmp::Ordering::Less => { - let lower = this_key_range.start; - gallop(this_keys, &mut this_key_range, |x| x < that_key); - merged.extend_from_keys(&this, lower .. this_key_range.start); - }, - std::cmp::Ordering::Equal => { - // keys are equal; must make a bespoke vals list. - // only push the key if merged.vals.values.len() advances. - let updates_len = merged.upds.borrow().values.len(); - let mut this_upd_range = this.upds_bounds(this_key_range.start .. this_key_range.start+1); - let mut that_upd_range = that.upds_bounds(that_key_range.start .. that_key_range.start+1); - - while !this_upd_range.is_empty() && !that_upd_range.is_empty() { - let (this_time, this_diff) = this_upds.values.get(this_upd_range.start); - let (that_time, that_diff) = that_upds.values.get(that_upd_range.start); - match this_time.cmp(&that_time) { - std::cmp::Ordering::Less => { - let lower = this_upd_range.start; - gallop(this_upds.values.0, &mut this_upd_range, |x| x < that_time); - merged.upds.as_mut().values.extend_from_self(this_upds.values, lower .. this_upd_range.start); - }, - std::cmp::Ordering::Equal => { - // times are equal; must add diffs. - this_sum.copy_from(this_diff); - that_sum.copy_from(that_diff); - this_sum.plus_equals(&that_sum); - if !this_sum.is_zero() { merged.upds.as_mut().values.push((this_time, &this_sum)); } - // Advance the update ranges by one. - this_upd_range.start += 1; - that_upd_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_upd_range.start; - gallop(that_upds.values.0, &mut that_upd_range, |x| x < this_time); - merged.upds.as_mut().values.extend_from_self(that_upds.values, lower .. that_upd_range.start); - }, - } - } - // Extend with the remaining this and that updates. - merged.upds.as_mut().values.extend_from_self(this_upds.values, this_upd_range); - merged.upds.as_mut().values.extend_from_self(that_upds.values, that_upd_range); - // Seal the values and push the key. - if merged.upds.borrow().values.len() > updates_len { - let temp_len = merged.upds.borrow().values.len() as u64; - merged.upds.as_mut().bounds.push(temp_len); - merged.keys.as_mut().push(this_key); - } - // Advance the key ranges by one. - this_key_range.start += 1; - that_key_range.start += 1; - }, - std::cmp::Ordering::Greater => { - let lower = that_key_range.start; - gallop(that_keys, &mut that_key_range, |x| x < this_key); - merged.extend_from_keys(&that, lower .. that_key_range.start); - }, - } - } - // Extend with the remaining this and that keys. - merged.extend_from_keys(&this, this_key_range); - merged.extend_from_keys(&that, that_key_range); - - merged - } - - #[inline(never)] - fn split(&mut self, frontier: AntichainRef) -> Self { - // Unfortunately the times are at the leaves, so there can be no bulk copying. - - use crate::Column; - use columnar::{ContainerOf, Vecs}; - - let mut ship_keys: ContainerOf = Default::default(); - let mut ship_upds: Vecs<(ContainerOf, ContainerOf)> = Default::default(); - let mut keep_keys: ContainerOf = Default::default(); - let mut keep_upds: Vecs<(ContainerOf, ContainerOf)> = Default::default(); - - let mut time = U::Time::default(); - for key_idx in 0 .. self.keys.borrow().len() { - let key = self.keys.borrow().get(key_idx); - let keep_upds_len = keep_upds.borrow().values.len(); - let ship_upds_len = ship_upds.borrow().values.len(); - for upd_idx in self.upds_bounds(key_idx..key_idx+1) { - let (t, diff) = self.upds.borrow().values.get(upd_idx); - time.copy_from(t); - if frontier.less_equal(&time) { - keep_upds.values.push((t, diff)); - } - else { - ship_upds.values.push((t, diff)); - } - } - if keep_upds.borrow().values.len() > keep_upds_len { - keep_upds.bounds.push(keep_upds.borrow().values.len() as u64); - keep_keys.push(key); - } - if ship_upds.borrow().values.len() > ship_upds_len { - ship_upds.bounds.push(ship_upds.borrow().values.len() as u64); - ship_keys.push(key); - } - } - - self.keys = Column::Typed(keep_keys); - self.upds = Column::Typed(keep_upds); - - // *self = keep; - // ship - Self { - keys: Column::Typed(ship_keys), - upds: Column::Typed(ship_upds), - } - } - - fn lower(&self, frontier: &mut Antichain) { - use columnar::Columnar; - let mut times = self.upds.borrow().values.0.into_index_iter(); - if let Some(time_ref) = times.next() { - let mut time = ::into_owned(time_ref); - frontier.insert_ref(&time); - for time_ref in times { - ::copy_from(&mut time, time_ref); - frontier.insert_ref(&time); - } - } - } - } - - #[inline(always)] - pub(crate) fn gallop(input: TC, range: &mut Range, mut cmp: impl FnMut(::Ref) -> bool) { - // if empty input, or already >= element, return - if !Range::::is_empty(range) && cmp(input.get(range.start)) { - let mut step = 1; - while range.start + step < range.end && cmp(input.get(range.start + step)) { - range.start += step; - step <<= 1; - } - - step >>= 1; - while step > 0 { - if range.start + step < range.end && cmp(input.get(range.start + step)) { - range.start += step; - } - step >>= 1; - } - - range.start += 1; - } - } - } - - use builder::val::ValMirror; - use builder::key::KeyMirror; - pub mod builder { - - pub mod val { - - use differential_dataflow::trace::implementations::ord_neu::{Vals, Upds}; - use differential_dataflow::trace::implementations::ord_neu::val_batch::{OrdValBatch, OrdValStorage}; - use differential_dataflow::trace::Description; - - use crate::ValStorage; - use crate::layout::ColumnarUpdate as Update; - use crate::layout::ColumnarLayout as Layout; - use crate::arrangement::Coltainer; - - use differential_dataflow::trace::implementations::OffsetList; - fn vec_u64_to_offset_list(list: Vec) -> OffsetList { - let mut output = OffsetList::with_capacity(list.len()); - output.push(0); - for item in list { output.push(item as usize); } - output - } - - pub struct ValMirror { marker: std::marker::PhantomData } - impl differential_dataflow::trace::Builder for ValMirror { - type Time = U::Time; - type Input = ValStorage; - type Output = OrdValBatch>; - - fn with_capacity(_keys: usize, _vals: usize, _upds: usize) -> Self { Self { marker: std::marker::PhantomData } } - fn push(&mut self, _chunk: &mut Self::Input) { unimplemented!() } - fn done(self, _description: Description) -> Self::Output { unimplemented!() } - fn seal(chain: &mut Vec, description: Description) -> Self::Output { - if chain.len() == 0 { - let storage = OrdValStorage { - keys: Default::default(), - vals: Default::default(), - upds: Default::default(), - }; - OrdValBatch { storage, description, updates: 0 } - } - else if chain.len() == 1 { - use columnar::Len; - let storage = chain.pop().unwrap(); - let updates = storage.upds.len(); - let storage = OrdValStorage { - keys: Coltainer { container: storage.keys }, - vals: Vals { - offs: vec_u64_to_offset_list(storage.vals.bounds), - vals: Coltainer { container: storage.vals.values }, - }, - upds: Upds { - offs: vec_u64_to_offset_list(storage.upds.bounds), - times: Coltainer { container: storage.upds.values.0 }, - diffs: Coltainer { container: storage.upds.values.1 }, - }, - }; - OrdValBatch { storage, description, updates } - } - else { - println!("chain length: {:?}", chain.len()); - unimplemented!() - } - } - } - } - - pub mod key { - - use differential_dataflow::trace::implementations::ord_neu::Upds; - use differential_dataflow::trace::implementations::ord_neu::key_batch::{OrdKeyBatch, OrdKeyStorage}; - use differential_dataflow::trace::Description; - - use crate::KeyStorage; - use crate::layout::ColumnarUpdate as Update; - use crate::layout::ColumnarLayout as Layout; - use crate::arrangement::Coltainer; - - use differential_dataflow::trace::implementations::OffsetList; - fn vec_u64_to_offset_list(list: Vec) -> OffsetList { - let mut output = OffsetList::with_capacity(list.len()); - output.push(0); - for item in list { output.push(item as usize); } - output - } - - pub struct KeyMirror { marker: std::marker::PhantomData } - impl> differential_dataflow::trace::Builder for KeyMirror { - type Time = U::Time; - type Input = KeyStorage; - type Output = OrdKeyBatch>; - - fn with_capacity(_keys: usize, _vals: usize, _upds: usize) -> Self { Self { marker: std::marker::PhantomData } } - fn push(&mut self, _chunk: &mut Self::Input) { unimplemented!() } - fn done(self, _description: Description) -> Self::Output { unimplemented!() } - fn seal(chain: &mut Vec, description: Description) -> Self::Output { - if chain.len() == 0 { - let storage = OrdKeyStorage { - keys: Default::default(), - upds: Default::default(), - }; - OrdKeyBatch { storage, description, updates: 0, value: OrdKeyBatch::>::create_value() } - } - else if chain.len() == 1 { - use columnar::Len; - let storage = chain.pop().unwrap(); - let updates = storage.upds.borrow().len(); - let upds = storage.upds.into_typed(); - let storage = OrdKeyStorage { - keys: Coltainer { container: storage.keys.into_typed() }, - upds: Upds { - offs: vec_u64_to_offset_list(upds.bounds), - times: Coltainer { container: upds.values.0 }, - diffs: Coltainer { container: upds.values.1 }, - }, - }; - OrdKeyBatch { storage, description, updates, value: OrdKeyBatch::>::create_value() } - } - else { panic!("chain length: {:?} > 1", chain.len()); } - } - } - } - } -} diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs new file mode 100644 index 000000000..ced700bec --- /dev/null +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -0,0 +1,1491 @@ +//! Columnar container infrastructure for differential dataflow. +//! +//! Provides trie-structured update storage (`Updates`, `RecordedUpdates`), +//! columnar arrangement types (`ValSpine`, `ValBatcher`, `ValBuilder`), +//! container traits for iterative scopes (`Enter`, `Leave`, `Negate`, `ResultsIn`), +//! exchange distribution (`ValPact`), and operators (`join_function`, `leave_dynamic`). +//! +//! Include via `#[path = "columnar_support.rs"] mod columnar_support;` + +#![allow(dead_code, unused_imports)] + +pub use layout::{ColumnarLayout, ColumnarUpdate}; +pub mod layout { + + use std::fmt::Debug; + use columnar::Columnar; + use differential_dataflow::trace::implementations::{Layout, OffsetList}; + use differential_dataflow::difference::Semigroup; + use differential_dataflow::lattice::Lattice; + use timely::progress::Timestamp; + + /// A layout based on columnar + pub struct ColumnarLayout { + phantom: std::marker::PhantomData, + } + + impl ColumnarUpdate for (K, V, T, R) + where + K: Columnar + Debug + Ord + Clone + 'static, + V: Columnar + Debug + Ord + Clone + 'static, + T: Columnar + Debug + Ord + Default + Clone + Lattice + Timestamp, + R: Columnar + Debug + Ord + Default + Semigroup + 'static, + { + type Key = K; + type Val = V; + type Time = T; + type Diff = R; + } + + use crate::arrangement::Coltainer; + impl Layout for ColumnarLayout { + type KeyContainer = Coltainer; + type ValContainer = Coltainer; + type TimeContainer = Coltainer; + type DiffContainer = Coltainer; + type OffsetContainer = OffsetList; + } + + /// A type that names constituent update types. + /// + /// We will use their associated `Columnar::Container` + pub trait ColumnarUpdate : Debug + 'static { + type Key: Columnar + Debug + Ord + Clone + 'static; + type Val: Columnar + Debug + Ord + Clone + 'static; + type Time: Columnar + Debug + Ord + Default + Clone + Lattice + Timestamp; + type Diff: Columnar + Debug + Ord + Default + Semigroup + 'static; + } + + /// A container whose references can be ordered. + pub trait OrdContainer : for<'a> columnar::Container : Ord> { } + impl columnar::Container : Ord>> OrdContainer for C { } + +} + +pub use updates::Updates; + +/// A thin wrapper around `Updates` that tracks the pre-consolidation record count +/// for timely's exchange accounting. This wrapper is the stream container type; +/// the `TrieChunker` strips it, passing bare `Updates` into the merge batcher. +pub struct RecordedUpdates { + pub updates: Updates, + pub records: usize, +} + +impl Default for RecordedUpdates { + fn default() -> Self { Self { updates: Default::default(), records: 0 } } +} + +impl Clone for RecordedUpdates { + fn clone(&self) -> Self { Self { updates: self.updates.clone(), records: self.records } } +} + +impl timely::Accountable for RecordedUpdates { + #[inline] fn record_count(&self) -> i64 { self.records as i64 } +} + +impl timely::dataflow::channels::ContainerBytes for RecordedUpdates { + fn from_bytes(_bytes: timely::bytes::arc::Bytes) -> Self { unimplemented!() } + fn length_in_bytes(&self) -> usize { unimplemented!() } + fn into_bytes(&self, _writer: &mut W) { unimplemented!() } +} + +// Container trait impls for RecordedUpdates, enabling iterative scopes. +mod container_impls { + use columnar::{Borrow, Columnar, Index, Len, Push}; + use timely::progress::{Timestamp, timestamp::Refines}; + use differential_dataflow::difference::Abelian; + use differential_dataflow::collection::containers::{Negate, Enter, Leave, ResultsIn}; + + use crate::layout::ColumnarUpdate as Update; + use crate::{RecordedUpdates, Updates}; + + impl> Negate for RecordedUpdates { + fn negate(mut self) -> Self { + let len = self.updates.diffs.values.len(); + let mut new_diffs = <::Container as Default>::default(); + let mut owned = U::Diff::default(); + for i in 0..len { + columnar::Columnar::copy_from(&mut owned, self.updates.diffs.values.borrow().get(i)); + owned.negate(); + new_diffs.push(&owned); + } + self.updates.diffs.values = new_diffs; + self + } + } + + impl Enter for RecordedUpdates<(K, V, T1, R)> + where + (K, V, T1, R): Update, + (K, V, T2, R): Update, + T1: Timestamp + Columnar + Default + Clone, + T2: Refines + Columnar + Default + Clone, + K: Columnar, V: Columnar, R: Columnar, + { + type InnerContainer = RecordedUpdates<(K, V, T2, R)>; + fn enter(self) -> Self::InnerContainer { + // Rebuild the time column; everything else moves as-is. + let mut new_times = <::Container as Default>::default(); + let mut t1_owned = T1::default(); + for i in 0..self.updates.times.values.len() { + Columnar::copy_from(&mut t1_owned, self.updates.times.values.borrow().get(i)); + let t2 = T2::to_inner(t1_owned.clone()); + new_times.push(&t2); + } + RecordedUpdates { + updates: Updates { + keys: self.updates.keys, + vals: self.updates.vals, + times: crate::updates::Lists { values: new_times, bounds: self.updates.times.bounds }, + diffs: self.updates.diffs, + }, + records: self.records, + } + } + } + + impl Leave for RecordedUpdates<(K, V, T1, R)> + where + (K, V, T1, R): Update, + (K, V, T2, R): Update, + T1: Refines + Columnar + Default + Clone, + T2: Timestamp + Columnar + Default + Clone, + K: Columnar, V: Columnar, R: Columnar, + { + type OuterContainer = RecordedUpdates<(K, V, T2, R)>; + fn leave(self) -> Self::OuterContainer { + // Flatten, convert times, and reconsolidate via consolidate. + // Leave can collapse distinct T1 times to the same T2 time, + // so the trie must be rebuilt with consolidation. + let mut flat = Updates::<(K, V, T2, R)>::default(); + let mut t1_owned = T1::default(); + for (k, v, t, d) in self.updates.iter() { + Columnar::copy_from(&mut t1_owned, t); + let t2: T2 = t1_owned.clone().to_outer(); + flat.push((k, v, &t2, d)); + } + RecordedUpdates { + updates: flat.consolidate(), + records: self.records, + } + } + } + + impl ResultsIn<::Summary> for RecordedUpdates { + fn results_in(self, step: &::Summary) -> Self { + use timely::progress::PathSummary; + // Apply results_in to each time; drop updates whose time maps to None. + // This must rebuild the trie since some entries may be removed. + let mut output = Updates::::default(); + let mut time_owned = U::Time::default(); + for (k, v, t, d) in self.updates.iter() { + Columnar::copy_from(&mut time_owned, t); + if let Some(new_time) = step.results_in(&time_owned) { + output.push((k, v, &new_time, d)); + } + } + RecordedUpdates { updates: output, records: self.records } + } + } +} + +pub use column_builder::ValBuilder as ValColBuilder; +mod column_builder { + + use std::collections::VecDeque; + use columnar::{Columnar, Clear, Len, Push}; + + use crate::layout::ColumnarUpdate as Update; + use crate::{Updates, RecordedUpdates}; + + type TupleContainer = <(::Key, ::Val, ::Time, ::Diff) as Columnar>::Container; + + /// A container builder that produces `RecordedUpdates` (sorted, consolidated trie + record count). + pub struct ValBuilder { + /// Container that we're writing to. + current: TupleContainer, + /// Empty allocation. + empty: Option>, + /// Completed containers pending to be sent. + pending: VecDeque>, + } + + use timely::container::PushInto; + impl PushInto for ValBuilder where TupleContainer : Push { + #[inline] + fn push_into(&mut self, item: T) { + self.current.push(item); + if self.current.len() > 1024 * 1024 { + use columnar::{Borrow, Index}; + let records = self.current.len(); + let mut refs = self.current.borrow().into_index_iter().collect::>(); + refs.sort(); + let updates = Updates::form(refs.into_iter()); + self.pending.push_back(RecordedUpdates { updates, records }); + self.current.clear(); + } + } + } + + impl Default for ValBuilder { + fn default() -> Self { + ValBuilder { + current: Default::default(), + empty: None, + pending: Default::default(), + } + } + } + + use timely::container::{ContainerBuilder, LengthPreservingContainerBuilder}; + impl ContainerBuilder for ValBuilder { + type Container = RecordedUpdates; + + #[inline] + fn extract(&mut self) -> Option<&mut Self::Container> { + if let Some(container) = self.pending.pop_front() { + self.empty = Some(container); + self.empty.as_mut() + } else { + None + } + } + + #[inline] + fn finish(&mut self) -> Option<&mut Self::Container> { + if !self.current.is_empty() { + use columnar::{Borrow, Index}; + let records = self.current.len(); + let mut refs = self.current.borrow().into_index_iter().collect::>(); + refs.sort(); + let updates = Updates::form(refs.into_iter()); + self.pending.push_back(RecordedUpdates { updates, records }); + self.current.clear(); + } + self.empty = self.pending.pop_front(); + self.empty.as_mut() + } + } + + impl LengthPreservingContainerBuilder for ValBuilder { } + +} + +pub use distributor::ValPact; +mod distributor { + + use std::rc::Rc; + + use columnar::{Borrow, Index, Len}; + use timely::logging::TimelyLogger; + use timely::dataflow::channels::pushers::{Exchange, exchange::Distributor}; + use timely::dataflow::channels::Message; + use timely::dataflow::channels::pact::{LogPuller, LogPusher, ParallelizationContract}; + use timely::progress::Timestamp; + use timely::worker::AsWorker; + + use crate::layout::ColumnarUpdate as Update; + use crate::{Updates, RecordedUpdates}; + + pub struct ValDistributor { + marker: std::marker::PhantomData, + hashfunc: H, + pre_lens: Vec, + } + + impl FnMut(columnar::Ref<'a, U::Key>)->u64> Distributor> for ValDistributor { + // TODO: For unsorted Updates (stride-1 outer keys), each key is its own outer group, + // so the per-group pre_lens snapshot and seal check costs O(keys × workers). Should + // either batch keys by destination first, or detect stride-1 outer bounds and use a + // simpler single-pass partitioning that seals once at the end. + fn partition>>>(&mut self, container: &mut RecordedUpdates, time: &T, pushers: &mut [P]) { + use crate::updates::child_range; + + let keys_b = container.updates.keys.borrow(); + let mut outputs: Vec> = (0..pushers.len()).map(|_| Updates::default()).collect(); + + // Each outer key group becomes a separate run in the destination. + for outer in 0..Len::len(&keys_b) { + self.pre_lens.clear(); + self.pre_lens.extend(outputs.iter().map(|o| o.keys.values.len())); + for k in child_range(keys_b.bounds, outer) { + let key = keys_b.values.get(k); + let idx = ((self.hashfunc)(key) as usize) % pushers.len(); + outputs[idx].extend_from_keys(&container.updates, k..k+1); + } + for (output, &pre) in outputs.iter_mut().zip(self.pre_lens.iter()) { + if output.keys.values.len() > pre { + output.keys.bounds.push(output.keys.values.len() as u64); + } + } + } + + // Distribute the input's record count across non-empty outputs. + let total_records = container.records; + let non_empty: usize = outputs.iter().filter(|o| !o.keys.values.is_empty()).count(); + let mut first_records = total_records.saturating_sub(non_empty.saturating_sub(1)); + for (pusher, output) in pushers.iter_mut().zip(outputs) { + if !output.keys.values.is_empty() { + let recorded = RecordedUpdates { updates: output, records: first_records }; + first_records = 1; + let mut recorded = recorded; + Message::push_at(&mut recorded, time.clone(), pusher); + } + } + } + fn flush>>>(&mut self, _time: &T, _pushers: &mut [P]) { } + fn relax(&mut self) { } + } + + pub struct ValPact { pub hashfunc: H } + + impl ParallelizationContract> for ValPact + where + T: Timestamp, + U: Update, + H: for<'a> FnMut(columnar::Ref<'a, U::Key>)->u64 + 'static, + { + type Pusher = Exchange< + T, + LogPusher>>>>, + ValDistributor + >; + type Puller = LogPuller>>>>; + + fn connect(self, allocator: &mut A, identifier: usize, address: Rc<[usize]>, logging: Option) -> (Self::Pusher, Self::Puller) { + let (senders, receiver) = allocator.allocate::>>(identifier, address); + let senders = senders.into_iter().enumerate().map(|(i,x)| LogPusher::new(x, allocator.index(), i, identifier, logging.clone())).collect::>(); + let distributor = ValDistributor { + marker: std::marker::PhantomData, + hashfunc: self.hashfunc, + pre_lens: Vec::new(), + }; + (Exchange::new(senders, distributor), LogPuller::new(receiver, allocator.index(), identifier, logging.clone())) + } + } +} + +pub use arrangement::{ValBatcher, ValBuilder, ValSpine}; +pub mod arrangement { + + use std::rc::Rc; + use differential_dataflow::trace::implementations::ord_neu::OrdValBatch; + use differential_dataflow::trace::rc_blanket_impls::RcBuilder; + use differential_dataflow::trace::implementations::spine_fueled::Spine; + + use crate::layout::ColumnarLayout; + + /// A trace implementation backed by columnar storage. + pub type ValSpine = Spine>>>; + /// A batcher for columnar storage. + pub type ValBatcher = ValBatcher2<(K,V,T,R)>; + /// A builder for columnar storage. + pub type ValBuilder = RcBuilder>; + + /// A batch container implementation for Coltainer. + pub use batch_container::Coltainer; + pub mod batch_container { + + use columnar::{Borrow, Columnar, Container, Clear, Push, Index, Len}; + use differential_dataflow::trace::implementations::BatchContainer; + + /// Container, anchored by `C` to provide an owned type. + pub struct Coltainer { + pub container: C::Container, + } + + impl Default for Coltainer { + fn default() -> Self { Self { container: Default::default() } } + } + + impl BatchContainer for Coltainer where for<'a> columnar::Ref<'a, C> : Ord { + + type ReadItem<'a> = columnar::Ref<'a, C>; + type Owned = C; + + #[inline(always)] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { C::into_owned(item) } + #[inline(always)] fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.copy_from(item) } + + #[inline(always)] fn push_ref(&mut self, item: Self::ReadItem<'_>) { self.container.push(item) } + #[inline(always)] fn push_own(&mut self, item: &Self::Owned) { self.container.push(item) } + + /// Clears the container. May not release resources. + fn clear(&mut self) { self.container.clear() } + + /// Creates a new container with sufficient capacity. + fn with_capacity(_size: usize) -> Self { Self::default() } + /// Creates a new container with sufficient capacity. + fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { + Self { + container: ::Container::with_capacity_for([cont1.container.borrow(), cont2.container.borrow()].into_iter()), + } + } + + /// Converts a read item into one with a narrower lifetime. + #[inline(always)] fn reborrow<'b, 'a: 'b>(item: Self::ReadItem<'a>) -> Self::ReadItem<'b> { columnar::ContainerOf::::reborrow_ref(item) } + + /// Reference to the element at this position. + #[inline(always)] fn index(&self, index: usize) -> Self::ReadItem<'_> { self.container.borrow().get(index) } + + #[inline(always)] fn len(&self) -> usize { self.container.len() } + } + } + + use crate::{Updates, RecordedUpdates}; + use differential_dataflow::trace::implementations::merge_batcher::{MergeBatcher, InternalMerger}; + type ValBatcher2 = MergeBatcher, TrieChunker, InternalMerger>>; + + /// A chunker that unwraps `RecordedUpdates` into bare `Updates` for the merge batcher. + /// The `records` accounting is discarded here — it has served its purpose for exchange. + /// + /// IMPORTANT: This chunker assumes the input `Updates` are sorted and consolidated + /// (as produced by `ValColBuilder::form`). The downstream `InternalMerge` relies on + /// this invariant. If `RecordedUpdates` could carry unsorted data (e.g. from a `map`), + /// we would need either a sorting chunker for that case, or a type-level distinction + /// (e.g. `RecordedUpdates` vs `RecordedUpdates`) to + /// route to the right chunker. + pub struct TrieChunker { + ready: std::collections::VecDeque>, + empty: Option>, + } + + impl Default for TrieChunker { + fn default() -> Self { Self { ready: Default::default(), empty: None } } + } + + impl<'a, U: crate::layout::ColumnarUpdate> timely::container::PushInto<&'a mut RecordedUpdates> for TrieChunker { + fn push_into(&mut self, container: &'a mut RecordedUpdates) { + self.ready.push_back(std::mem::take(&mut container.updates)); + } + } + + impl timely::container::ContainerBuilder for TrieChunker { + type Container = Updates; + fn extract(&mut self) -> Option<&mut Self::Container> { + if let Some(ready) = self.ready.pop_front() { + self.empty = Some(ready); + self.empty.as_mut() + } else { + None + } + } + fn finish(&mut self) -> Option<&mut Self::Container> { + self.empty = self.ready.pop_front(); + self.empty.as_mut() + } + } + + pub mod batcher { + + use std::ops::Range; + use columnar::{Borrow, Columnar, Container, Index, Len, Push}; + use differential_dataflow::difference::{Semigroup, IsZero}; + use timely::progress::frontier::{Antichain, AntichainRef}; + use differential_dataflow::trace::implementations::merge_batcher::container::InternalMerge; + + use crate::ColumnarUpdate as Update; + use crate::Updates; + + impl timely::container::SizableContainer for Updates { + fn at_capacity(&self) -> bool { + use columnar::Len; + self.diffs.values.len() >= 64 * 1024 + } + fn ensure_capacity(&mut self, _stash: &mut Option) { } + } + + impl InternalMerge for Updates { + + type TimeOwned = U::Time; + + fn len(&self) -> usize { self.diffs.values.len() } + fn clear(&mut self) { *self = Self::default(); } + + #[inline(never)] + fn merge_from(&mut self, others: &mut [Self], positions: &mut [usize]) { + match others.len() { + 0 => {}, + 1 => { + // Bulk copy: take remaining keys from position onward. + let other = &mut others[0]; + let pos = &mut positions[0]; + if self.keys.values.len() == 0 && *pos == 0 { + std::mem::swap(self, other); + return; + } + let other_len = other.keys.values.len(); + self.extend_from_keys(other, *pos .. other_len); + *pos = other_len; + }, + 2 => { + let mut this_sum = U::Diff::default(); + let mut that_sum = U::Diff::default(); + + let (left, right) = others.split_at_mut(1); + let this = &left[0]; + let that = &right[0]; + let this_keys = this.keys.values.borrow(); + let that_keys = that.keys.values.borrow(); + let mut this_key_range = positions[0] .. this_keys.len(); + let mut that_key_range = positions[1] .. that_keys.len(); + + while !this_key_range.is_empty() && !that_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { + let this_key = this_keys.get(this_key_range.start); + let that_key = that_keys.get(that_key_range.start); + match this_key.cmp(&that_key) { + std::cmp::Ordering::Less => { + let lower = this_key_range.start; + gallop(this_keys, &mut this_key_range, |x| x < that_key); + self.extend_from_keys(this, lower .. this_key_range.start); + }, + std::cmp::Ordering::Equal => { + let values_len = self.vals.values.len(); + let mut this_val_range = this.vals_bounds(this_key_range.start .. this_key_range.start+1); + let mut that_val_range = that.vals_bounds(that_key_range.start .. that_key_range.start+1); + while !this_val_range.is_empty() && !that_val_range.is_empty() { + let this_val = this.vals.values.borrow().get(this_val_range.start); + let that_val = that.vals.values.borrow().get(that_val_range.start); + match this_val.cmp(&that_val) { + std::cmp::Ordering::Less => { + let lower = this_val_range.start; + gallop(this.vals.values.borrow(), &mut this_val_range, |x| x < that_val); + self.extend_from_vals(this, lower .. this_val_range.start); + }, + std::cmp::Ordering::Equal => { + let updates_len = self.times.values.len(); + let mut this_time_range = this.times_bounds(this_val_range.start .. this_val_range.start+1); + let mut that_time_range = that.times_bounds(that_val_range.start .. that_val_range.start+1); + while !this_time_range.is_empty() && !that_time_range.is_empty() { + let this_time = this.times.values.borrow().get(this_time_range.start); + let this_diff = this.diffs.values.borrow().get(this_time_range.start); + let that_time = that.times.values.borrow().get(that_time_range.start); + let that_diff = that.diffs.values.borrow().get(that_time_range.start); + match this_time.cmp(&that_time) { + std::cmp::Ordering::Less => { + let lower = this_time_range.start; + gallop(this.times.values.borrow(), &mut this_time_range, |x| x < that_time); + self.times.values.extend_from_self(this.times.values.borrow(), lower .. this_time_range.start); + self.diffs.extend_from_self(this.diffs.borrow(), lower .. this_time_range.start); + }, + std::cmp::Ordering::Equal => { + this_sum.copy_from(this_diff); + that_sum.copy_from(that_diff); + this_sum.plus_equals(&that_sum); + if !this_sum.is_zero() { + self.times.values.push(this_time); + self.diffs.values.push(&this_sum); + self.diffs.bounds.push(self.diffs.values.len() as u64); + } + this_time_range.start += 1; + that_time_range.start += 1; + }, + std::cmp::Ordering::Greater => { + let lower = that_time_range.start; + gallop(that.times.values.borrow(), &mut that_time_range, |x| x < this_time); + self.times.values.extend_from_self(that.times.values.borrow(), lower .. that_time_range.start); + self.diffs.extend_from_self(that.diffs.borrow(), lower .. that_time_range.start); + }, + } + } + // Remaining from this side + if !this_time_range.is_empty() { + self.times.values.extend_from_self(this.times.values.borrow(), this_time_range.clone()); + self.diffs.extend_from_self(this.diffs.borrow(), this_time_range.clone()); + } + // Remaining from that side + if !that_time_range.is_empty() { + self.times.values.extend_from_self(that.times.values.borrow(), that_time_range.clone()); + self.diffs.extend_from_self(that.diffs.borrow(), that_time_range.clone()); + } + if self.times.values.len() > updates_len { + self.times.bounds.push(self.times.values.len() as u64); + self.vals.values.push(this_val); + } + this_val_range.start += 1; + that_val_range.start += 1; + }, + std::cmp::Ordering::Greater => { + let lower = that_val_range.start; + gallop(that.vals.values.borrow(), &mut that_val_range, |x| x < this_val); + self.extend_from_vals(that, lower .. that_val_range.start); + }, + } + } + self.extend_from_vals(this, this_val_range); + self.extend_from_vals(that, that_val_range); + if self.vals.values.len() > values_len { + self.vals.bounds.push(self.vals.values.len() as u64); + self.keys.values.push(this_key); + } + this_key_range.start += 1; + that_key_range.start += 1; + }, + std::cmp::Ordering::Greater => { + let lower = that_key_range.start; + gallop(that_keys, &mut that_key_range, |x| x < this_key); + self.extend_from_keys(that, lower .. that_key_range.start); + }, + } + } + // Copy remaining from whichever side has data, up to capacity. + while !this_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { + let lower = this_key_range.start; + this_key_range.start = this_key_range.end; // take all remaining + self.extend_from_keys(this, lower .. this_key_range.start); + } + while !that_key_range.is_empty() && !timely::container::SizableContainer::at_capacity(self) { + let lower = that_key_range.start; + that_key_range.start = that_key_range.end; + self.extend_from_keys(that, lower .. that_key_range.start); + } + positions[0] = this_key_range.start; + positions[1] = that_key_range.start; + }, + n => unimplemented!("{n}-way merge not supported"), + } + } + + fn extract( + &mut self, + upper: AntichainRef, + frontier: &mut Antichain, + keep: &mut Self, + ship: &mut Self, + ) { + let mut time = U::Time::default(); + for key_idx in 0 .. self.keys.values.len() { + let key = self.keys.values.borrow().get(key_idx); + let keep_vals_len = keep.vals.values.len(); + let ship_vals_len = ship.vals.values.len(); + for val_idx in self.vals_bounds(key_idx..key_idx+1) { + let val = self.vals.values.borrow().get(val_idx); + let keep_times_len = keep.times.values.len(); + let ship_times_len = ship.times.values.len(); + for time_idx in self.times_bounds(val_idx..val_idx+1) { + let t = self.times.values.borrow().get(time_idx); + let diff = self.diffs.values.borrow().get(time_idx); + time.copy_from(t); + if upper.less_equal(&time) { + frontier.insert_ref(&time); + keep.times.values.push(t); + keep.diffs.values.push(diff); + keep.diffs.bounds.push(keep.diffs.values.len() as u64); + } + else { + ship.times.values.push(t); + ship.diffs.values.push(diff); + ship.diffs.bounds.push(ship.diffs.values.len() as u64); + } + } + if keep.times.values.len() > keep_times_len { + keep.times.bounds.push(keep.times.values.len() as u64); + keep.vals.values.push(val); + } + if ship.times.values.len() > ship_times_len { + ship.times.bounds.push(ship.times.values.len() as u64); + ship.vals.values.push(val); + } + } + if keep.vals.values.len() > keep_vals_len { + keep.vals.bounds.push(keep.vals.values.len() as u64); + keep.keys.values.push(key); + } + if ship.vals.values.len() > ship_vals_len { + ship.vals.bounds.push(ship.vals.values.len() as u64); + ship.keys.values.push(key); + } + } + } + } + + #[inline(always)] + pub(crate) fn gallop(input: TC, range: &mut Range, mut cmp: impl FnMut(::Ref) -> bool) { + // if empty input, or already >= element, return + if !Range::::is_empty(range) && cmp(input.get(range.start)) { + let mut step = 1; + while range.start + step < range.end && cmp(input.get(range.start + step)) { + range.start += step; + step <<= 1; + } + + step >>= 1; + while step > 0 { + if range.start + step < range.end && cmp(input.get(range.start + step)) { + range.start += step; + } + step >>= 1; + } + + range.start += 1; + } + } + } + + use builder::ValMirror; + pub mod builder { + + use differential_dataflow::trace::implementations::ord_neu::{Vals, Upds}; + use differential_dataflow::trace::implementations::ord_neu::val_batch::{OrdValBatch, OrdValStorage}; + use differential_dataflow::trace::Description; + + use crate::Updates; + use crate::layout::ColumnarUpdate as Update; + use crate::layout::ColumnarLayout as Layout; + use crate::arrangement::Coltainer; + + use columnar::{Borrow, IndexAs}; + use columnar::primitive::offsets::Strides; + use differential_dataflow::trace::implementations::OffsetList; + fn strides_to_offset_list(bounds: &Strides, count: usize) -> OffsetList { + let mut output = OffsetList::with_capacity(count); + output.push(0); + let bounds_b = bounds.borrow(); + for i in 0..count { + output.push(bounds_b.index_as(i) as usize); + } + output + } + + pub struct ValMirror { + current: Updates, + } + impl differential_dataflow::trace::Builder for ValMirror { + type Time = U::Time; + type Input = Updates; + type Output = OrdValBatch>; + + fn with_capacity(_keys: usize, _vals: usize, _upds: usize) -> Self { + Self { current: Updates::default() } + } + fn push(&mut self, chunk: &mut Self::Input) { + use columnar::Len; + let len = chunk.keys.values.len(); + if len > 0 { + self.current.extend_from_keys(chunk, 0..len); + } + } + fn done(self, description: Description) -> Self::Output { + let mut chain = if self.current.len() > 0 { + vec![self.current] + } else { + vec![] + }; + Self::seal(&mut chain, description) + } + fn seal(chain: &mut Vec, description: Description) -> Self::Output { + if chain.len() == 0 { + let storage = OrdValStorage { + keys: Default::default(), + vals: Default::default(), + upds: Default::default(), + }; + OrdValBatch { storage, description, updates: 0 } + } + else if chain.len() == 1 { + use columnar::Len; + let storage = chain.pop().unwrap(); + let updates = storage.diffs.values.len(); + let val_offs = strides_to_offset_list(&storage.vals.bounds, storage.keys.values.len()); + let time_offs = strides_to_offset_list(&storage.times.bounds, storage.vals.values.len()); + let storage = OrdValStorage { + keys: Coltainer { container: storage.keys.values }, + vals: Vals { + offs: val_offs, + vals: Coltainer { container: storage.vals.values }, + }, + upds: Upds { + offs: time_offs, + times: Coltainer { container: storage.times.values }, + diffs: Coltainer { container: storage.diffs.values }, + }, + }; + OrdValBatch { storage, description, updates } + } + else { + use columnar::Len; + let mut merged = chain.remove(0); + for other in chain.drain(..) { + let len = other.keys.values.len(); + if len > 0 { + merged.extend_from_keys(&other, 0..len); + } + } + chain.push(merged); + Self::seal(chain, description) + } + } + } + + } +} + +pub mod updates { + + use columnar::{Columnar, Container, ContainerOf, Vecs, Borrow, Index, IndexAs, Len, Push}; + use columnar::primitive::offsets::Strides; + use differential_dataflow::difference::{Semigroup, IsZero}; + + use crate::layout::ColumnarUpdate as Update; + + /// A `Vecs` using strided offsets. + pub type Lists = Vecs; + + /// Trie-structured update storage using columnar containers. + /// + /// Four nested layers of `Lists`: + /// - `keys`: lists of keys (outer lists are independent groups) + /// - `vals`: per-key, lists of vals + /// - `times`: per-val, lists of times + /// - `diffs`: per-time, lists of diffs (singletons when consolidated) + /// + /// A flat unsorted input has stride 1 at every level (one key per entry, + /// one val per key, one time per val, one diff per time). + /// A fully consolidated trie has a single outer key list, all lists sorted + /// and deduplicated, and singleton diff lists. + pub struct Updates { + pub keys: Lists>, + pub vals: Lists>, + pub times: Lists>, + pub diffs: Lists>, + } + + impl Default for Updates { + fn default() -> Self { + Self { + keys: Default::default(), + vals: Default::default(), + times: Default::default(), + diffs: Default::default(), + } + } + } + + impl std::fmt::Debug for Updates { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("Updates").finish() + } + } + + impl Clone for Updates { + fn clone(&self) -> Self { + Self { + keys: self.keys.clone(), + vals: self.vals.clone(), + times: self.times.clone(), + diffs: self.diffs.clone(), + } + } + } + + pub type Tuple = (::Key, ::Val, ::Time, ::Diff); + + /// Returns the value-index range for list `i` given cumulative bounds. + #[inline] + pub fn child_range>(bounds: B, i: usize) -> std::ops::Range { + let lower = if i == 0 { 0 } else { bounds.index_as(i - 1) as usize }; + let upper = bounds.index_as(i) as usize; + lower..upper + } + + impl Updates { + + pub fn vals_bounds(&self, key_range: std::ops::Range) -> std::ops::Range { + if !key_range.is_empty() { + let bounds = self.vals.bounds.borrow(); + let lower = if key_range.start == 0 { 0 } else { bounds.index_as(key_range.start - 1) as usize }; + let upper = bounds.index_as(key_range.end - 1) as usize; + lower..upper + } else { key_range } + } + pub fn times_bounds(&self, val_range: std::ops::Range) -> std::ops::Range { + if !val_range.is_empty() { + let bounds = self.times.bounds.borrow(); + let lower = if val_range.start == 0 { 0 } else { bounds.index_as(val_range.start - 1) as usize }; + let upper = bounds.index_as(val_range.end - 1) as usize; + lower..upper + } else { val_range } + } + pub fn diffs_bounds(&self, time_range: std::ops::Range) -> std::ops::Range { + if !time_range.is_empty() { + let bounds = self.diffs.bounds.borrow(); + let lower = if time_range.start == 0 { 0 } else { bounds.index_as(time_range.start - 1) as usize }; + let upper = bounds.index_as(time_range.end - 1) as usize; + lower..upper + } else { time_range } + } + + /// Copies `other[key_range]` into self, keys and all. + pub fn extend_from_keys(&mut self, other: &Self, key_range: std::ops::Range) { + self.keys.values.extend_from_self(other.keys.values.borrow(), key_range.clone()); + self.vals.extend_from_self(other.vals.borrow(), key_range.clone()); + let val_range = other.vals_bounds(key_range); + self.times.extend_from_self(other.times.borrow(), val_range.clone()); + let time_range = other.times_bounds(val_range); + self.diffs.extend_from_self(other.diffs.borrow(), time_range); + } + + /// Copies a range of vals (with their times and diffs) from `other` into self. + pub fn extend_from_vals(&mut self, other: &Self, val_range: std::ops::Range) { + self.vals.values.extend_from_self(other.vals.values.borrow(), val_range.clone()); + self.times.extend_from_self(other.times.borrow(), val_range.clone()); + let time_range = other.times_bounds(val_range); + self.diffs.extend_from_self(other.diffs.borrow(), time_range); + } + + /// Forms a consolidated `Updates` from sorted `(key, val, time, diff)` refs. + /// + /// Tracks a `prev` reference to the previous element. On each new element, + /// compares against `prev` to detect key/val/time changes. Only pushes + /// accumulated diffs when they are nonzero, and only emits times/vals/keys + /// that have at least one nonzero diff beneath them. + pub fn form<'a>(mut sorted: impl Iterator>>) -> Self { + + let mut output = Self::default(); + let mut diff_stash = U::Diff::default(); + let mut diff_temp = U::Diff::default(); + + if let Some(first) = sorted.next() { + + let mut prev = first; + Columnar::copy_from(&mut diff_stash, prev.3); + + for curr in sorted { + let key_differs = ContainerOf::::reborrow_ref(curr.0) != ContainerOf::::reborrow_ref(prev.0); + let val_differs = key_differs || ContainerOf::::reborrow_ref(curr.1) != ContainerOf::::reborrow_ref(prev.1); + let time_differs = val_differs || ContainerOf::::reborrow_ref(curr.2) != ContainerOf::::reborrow_ref(prev.2); + + if time_differs { + // Flush the accumulated diff for prev's (key, val, time). + if !diff_stash.is_zero() { + // We have a real update to emit. Push time (and val/key + // if this is the first time under them). + let times_len = output.times.values.len(); + let vals_len = output.vals.values.len(); + + if val_differs { + // Seal the previous val's time list, if any times were emitted. + if times_len > 0 { + output.times.bounds.push(times_len as u64); + } + if key_differs { + // Seal the previous key's val list, if any vals were emitted. + if vals_len > 0 { + output.vals.bounds.push(vals_len as u64); + } + output.keys.values.push(prev.0); + } + output.vals.values.push(prev.1); + } + output.times.values.push(prev.2); + output.diffs.values.push(&diff_stash); + output.diffs.bounds.push(output.diffs.values.len() as u64); + } + Columnar::copy_from(&mut diff_stash, curr.3); + } else { + // Same (key, val, time): accumulate diff. + Columnar::copy_from(&mut diff_temp, curr.3); + diff_stash.plus_equals(&diff_temp); + } + prev = curr; + } + + // Flush the final accumulated diff. + if !diff_stash.is_zero() { + let keys_len = output.keys.values.len(); + let vals_len = output.vals.values.len(); + let times_len = output.times.values.len(); + let need_key = keys_len == 0 || ContainerOf::::reborrow_ref(prev.0) != output.keys.values.borrow().get(keys_len - 1); + let need_val = need_key || vals_len == 0 || ContainerOf::::reborrow_ref(prev.1) != output.vals.values.borrow().get(vals_len - 1); + + if need_val { + if times_len > 0 { + output.times.bounds.push(times_len as u64); + } + if need_key { + if vals_len > 0 { + output.vals.bounds.push(vals_len as u64); + } + output.keys.values.push(prev.0); + } + output.vals.values.push(prev.1); + } + output.times.values.push(prev.2); + output.diffs.values.push(&diff_stash); + output.diffs.bounds.push(output.diffs.values.len() as u64); + } + + // Seal the final groups at each level. + if !output.times.values.is_empty() { + output.times.bounds.push(output.times.values.len() as u64); + } + if !output.vals.values.is_empty() { + output.vals.bounds.push(output.vals.values.len() as u64); + } + if !output.keys.values.is_empty() { + output.keys.bounds.push(output.keys.values.len() as u64); + } + } + + output + } + + /// Consolidates into canonical trie form: + /// single outer key list, all lists sorted and deduplicated, + /// diff lists are singletons (or absent if cancelled). + pub fn consolidate(self) -> Self { + + let Self { keys, vals, times, diffs } = self; + + let keys_b = keys.borrow(); + let vals_b = vals.borrow(); + let times_b = times.borrow(); + let diffs_b = diffs.borrow(); + + // Flatten to index tuples: [key_abs, val_abs, time_abs, diff_abs]. + let mut tuples: Vec<[usize; 4]> = Vec::new(); + for outer in 0..Len::len(&keys_b) { + for k in child_range(keys_b.bounds, outer) { + for v in child_range(vals_b.bounds, k) { + for t in child_range(times_b.bounds, v) { + for d in child_range(diffs_b.bounds, t) { + tuples.push([k, v, t, d]); + } + } + } + } + } + + // Sort by (key, val, time). Diff is payload. + tuples.sort_by(|a, b| { + keys_b.values.get(a[0]).cmp(&keys_b.values.get(b[0])) + .then_with(|| vals_b.values.get(a[1]).cmp(&vals_b.values.get(b[1]))) + .then_with(|| times_b.values.get(a[2]).cmp(×_b.values.get(b[2]))) + }); + + // Build consolidated output, bottom-up cancellation. + let mut output = Self::default(); + let mut diff_stash = U::Diff::default(); + let mut diff_temp = U::Diff::default(); + + let mut idx = 0; + while idx < tuples.len() { + let key_ref = keys_b.values.get(tuples[idx][0]); + let key_start_vals = output.vals.values.len(); + + // All entries with this key. + while idx < tuples.len() && keys_b.values.get(tuples[idx][0]) == key_ref { + let val_ref = vals_b.values.get(tuples[idx][1]); + let val_start_times = output.times.values.len(); + + // All entries with this (key, val). + while idx < tuples.len() + && keys_b.values.get(tuples[idx][0]) == key_ref + && vals_b.values.get(tuples[idx][1]) == val_ref + { + let time_ref = times_b.values.get(tuples[idx][2]); + + // Sum all diffs for this (key, val, time). + Columnar::copy_from(&mut diff_stash, diffs_b.values.get(tuples[idx][3])); + idx += 1; + while idx < tuples.len() + && keys_b.values.get(tuples[idx][0]) == key_ref + && vals_b.values.get(tuples[idx][1]) == val_ref + && times_b.values.get(tuples[idx][2]) == time_ref + { + Columnar::copy_from(&mut diff_temp, diffs_b.values.get(tuples[idx][3])); + diff_stash.plus_equals(&diff_temp); + idx += 1; + } + + // Emit time + singleton diff if nonzero. + if !diff_stash.is_zero() { + output.times.values.push(time_ref); + output.diffs.values.push(&diff_stash); + output.diffs.bounds.push(output.diffs.values.len() as u64); + } + } + + // Seal time list for this val; emit val if any times survived. + if output.times.values.len() > val_start_times { + output.times.bounds.push(output.times.values.len() as u64); + output.vals.values.push(val_ref); + } + } + + // Seal val list for this key; emit key if any vals survived. + if output.vals.values.len() > key_start_vals { + output.vals.bounds.push(output.vals.values.len() as u64); + output.keys.values.push(key_ref); + } + } + + // Seal the single outer key list. + if !output.keys.values.is_empty() { + output.keys.bounds.push(output.keys.values.len() as u64); + } + + output + } + + /// The number of leaf-level diff entries (total updates). + pub fn len(&self) -> usize { self.diffs.values.len() } + } + + /// Push a single flat update as a stride-1 entry. + /// + /// Each field is independently typed — columnar refs, `&Owned`, owned values, + /// or any other type the column container accepts via its `Push` impl. + impl Push<(KP, VP, TP, DP)> for Updates + where + ContainerOf: Push, + ContainerOf: Push, + ContainerOf: Push, + ContainerOf: Push, + { + fn push(&mut self, (key, val, time, diff): (KP, VP, TP, DP)) { + self.keys.values.push(key); + self.keys.bounds.push(self.keys.values.len() as u64); + self.vals.values.push(val); + self.vals.bounds.push(self.vals.values.len() as u64); + self.times.values.push(time); + self.times.bounds.push(self.times.values.len() as u64); + self.diffs.values.push(diff); + self.diffs.bounds.push(self.diffs.values.len() as u64); + } + } + + /// PushInto for the `((K, V), T, R)` shape that reduce_trace uses. + impl timely::container::PushInto<((U::Key, U::Val), U::Time, U::Diff)> for Updates { + fn push_into(&mut self, ((key, val), time, diff): ((U::Key, U::Val), U::Time, U::Diff)) { + self.push((&key, &val, &time, &diff)); + } + } + + impl Updates { + + /// Iterate all `(key, val, time, diff)` entries as refs. + pub fn iter(&self) -> impl Iterator, + columnar::Ref<'_, U::Val>, + columnar::Ref<'_, U::Time>, + columnar::Ref<'_, U::Diff>, + )> { + let keys_b = self.keys.borrow(); + let vals_b = self.vals.borrow(); + let times_b = self.times.borrow(); + let diffs_b = self.diffs.borrow(); + + (0..Len::len(&keys_b)) + .flat_map(move |outer| child_range(keys_b.bounds, outer)) + .flat_map(move |k| { + let key = keys_b.values.get(k); + child_range(vals_b.bounds, k).map(move |v| (key, v)) + }) + .flat_map(move |(key, v)| { + let val = vals_b.values.get(v); + child_range(times_b.bounds, v).map(move |t| (key, val, t)) + }) + .flat_map(move |(key, val, t)| { + let time = times_b.values.get(t); + child_range(diffs_b.bounds, t).map(move |d| (key, val, time, diffs_b.values.get(d))) + }) + } + } + + impl timely::Accountable for Updates { + #[inline] fn record_count(&self) -> i64 { Len::len(&self.diffs.values) as i64 } + } + + impl timely::dataflow::channels::ContainerBytes for Updates { + fn from_bytes(_bytes: timely::bytes::arc::Bytes) -> Self { unimplemented!() } + fn length_in_bytes(&self) -> usize { unimplemented!() } + fn into_bytes(&self, _writer: &mut W) { unimplemented!() } + } + + #[cfg(test)] + mod tests { + use super::*; + use columnar::Push; + + type TestUpdate = (u64, u64, u64, i64); + + fn collect(updates: &Updates) -> Vec<(u64, u64, u64, i64)> { + updates.iter().map(|(k, v, t, d)| (*k, *v, *t, *d)).collect() + } + + #[test] + fn test_push_and_consolidate_basic() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &100, &2)); + updates.push((&2, &20, &200, &5)); + assert_eq!(updates.len(), 3); + assert_eq!(collect(&updates.consolidate()), vec![(1, 10, 100, 3), (2, 20, 200, 5)]); + } + + #[test] + fn test_cancellation() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &3)); + updates.push((&1, &10, &100, &-3)); + updates.push((&2, &20, &200, &1)); + assert_eq!(collect(&updates.consolidate()), vec![(2, 20, 200, 1)]); + } + + #[test] + fn test_multiple_vals_and_times() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &200, &2)); + updates.push((&1, &20, &100, &3)); + updates.push((&1, &20, &100, &4)); + assert_eq!(collect(&updates.consolidate()), vec![(1, 10, 100, 1), (1, 10, 200, 2), (1, 20, 100, 7)]); + } + + #[test] + fn test_val_cancellation_propagates() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &5)); + updates.push((&1, &10, &100, &-5)); + updates.push((&1, &20, &100, &1)); + assert_eq!(collect(&updates.consolidate()), vec![(1, 20, 100, 1)]); + } + + #[test] + fn test_empty() { + let updates = Updates::::default(); + assert_eq!(collect(&updates.consolidate()), vec![]); + } + + #[test] + fn test_total_cancellation() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &100, &-1)); + assert_eq!(collect(&updates.consolidate()), vec![]); + } + + #[test] + fn test_unsorted_input() { + let mut updates = Updates::::default(); + updates.push((&3, &30, &300, &1)); + updates.push((&1, &10, &100, &2)); + updates.push((&2, &20, &200, &3)); + assert_eq!(collect(&updates.consolidate()), vec![(1, 10, 100, 2), (2, 20, 200, 3), (3, 30, 300, 1)]); + } + + #[test] + fn test_first_key_cancels() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &5)); + updates.push((&1, &10, &100, &-5)); + updates.push((&2, &20, &200, &3)); + assert_eq!(collect(&updates.consolidate()), vec![(2, 20, 200, 3)]); + } + + #[test] + fn test_middle_time_cancels() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &200, &2)); + updates.push((&1, &10, &200, &-2)); + updates.push((&1, &10, &300, &3)); + assert_eq!(collect(&updates.consolidate()), vec![(1, 10, 100, 1), (1, 10, 300, 3)]); + } + + #[test] + fn test_first_val_cancels() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &100, &-1)); + updates.push((&1, &20, &100, &5)); + assert_eq!(collect(&updates.consolidate()), vec![(1, 20, 100, 5)]); + } + + #[test] + fn test_interleaved_cancellations() { + let mut updates = Updates::::default(); + updates.push((&1, &10, &100, &1)); + updates.push((&1, &10, &100, &-1)); + updates.push((&2, &20, &200, &7)); + updates.push((&3, &30, &300, &4)); + updates.push((&3, &30, &300, &-4)); + assert_eq!(collect(&updates.consolidate()), vec![(2, 20, 200, 7)]); + } + } +} + +/// A columnar flat_map: iterates RecordedUpdates, calls logic per (key, val, time, diff), +/// joins output times with input times, multiplies output diffs with input diffs. +/// +/// This subsumes map, filter, negate, and enter_at for columnar collections. +pub fn join_function( + input: differential_dataflow::Collection>, + mut logic: L, +) -> differential_dataflow::Collection> +where + G: timely::dataflow::Scope, + G::Timestamp: differential_dataflow::lattice::Lattice, + U: layout::ColumnarUpdate>, + I: IntoIterator, + L: FnMut( + columnar::Ref<'_, U::Key>, + columnar::Ref<'_, U::Val>, + columnar::Ref<'_, U::Time>, + columnar::Ref<'_, U::Diff>, + ) -> I + 'static, +{ + use timely::dataflow::operators::generic::Operator; + use timely::dataflow::channels::pact::Pipeline; + use differential_dataflow::AsCollection; + use differential_dataflow::difference::Multiply; + use differential_dataflow::lattice::Lattice; + use columnar::Columnar; + + input + .inner + .unary::, _, _, _>(Pipeline, "JoinFunction", move |_, _| { + move |input, output| { + input.for_each(|time, data| { + let mut session = output.session_with_builder(&time); + for (k1, v1, t1, d1) in data.updates.iter() { + let t1o: U::Time = Columnar::into_owned(t1); + let d1o: U::Diff = Columnar::into_owned(d1); + for (k2, v2, t2, d2) in logic(k1, v1, t1, d1) { + let t3 = t2.join(&t1o); + let d3 = d2.multiply(&d1o); + session.give((&k2, &v2, &t3, &d3)); + } + } + }); + } + }) + .as_collection() +} + +type DynTime = timely::order::Product>; + +/// Leave a dynamic iterative scope, truncating PointStamp coordinates. +/// +/// Uses OperatorBuilder (not unary) for the custom input connection summary +/// that tells timely how the PointStamp is affected (retain `level - 1` coordinates). +/// +/// Consolidates after truncation since distinct PointStamp coordinates can collapse. +pub fn leave_dynamic( + input: differential_dataflow::Collection>, + level: usize, +) -> differential_dataflow::Collection> +where + G: timely::dataflow::Scope, + K: columnar::Columnar, + V: columnar::Columnar, + R: columnar::Columnar, + (K, V, DynTime, R): layout::ColumnarUpdate, +{ + use timely::dataflow::channels::pact::Pipeline; + use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; + use timely::dataflow::operators::generic::OutputBuilder; + use timely::order::Product; + use timely::progress::Antichain; + use timely::container::{ContainerBuilder, PushInto}; + use differential_dataflow::AsCollection; + use differential_dataflow::dynamic::pointstamp::{PointStamp, PointStampSummary}; + use columnar::Columnar; + + let mut builder = OperatorBuilder::new("LeaveDynamic".to_string(), input.inner.scope()); + let (output, stream) = builder.new_output(); + let mut output = OutputBuilder::from(output); + let mut op_input = builder.new_input_connection( + input.inner, + Pipeline, + [( + 0, + Antichain::from_elem(Product { + outer: Default::default(), + inner: PointStampSummary { + retain: Some(level - 1), + actions: Vec::new(), + }, + }), + )], + ); + + builder.build(move |_capability| { + let mut col_builder = ValColBuilder::<(K, V, DynTime, R)>::default(); + move |_frontier| { + let mut output = output.activate(); + op_input.for_each(|cap, data| { + // Truncate the capability's timestamp. + let mut new_time = cap.time().clone(); + let mut vec = std::mem::take(&mut new_time.inner).into_inner(); + vec.truncate(level - 1); + new_time.inner = PointStamp::new(vec); + let new_cap = cap.delayed(&new_time, 0); + // Push updates with truncated times into the builder. + // The builder's form call on flush sorts and consolidates, + // handling the duplicate times that truncation can produce. + // TODO: The input trie is already sorted; a streaming form + // that accepts pre-sorted, potentially-collapsing timestamps + // could avoid the re-sort inside the builder. + for (k, v, t, d) in data.updates.iter() { + let mut time: DynTime = Columnar::into_owned(t); + let mut inner_vec = std::mem::take(&mut time.inner).into_inner(); + inner_vec.truncate(level - 1); + time.inner = PointStamp::new(inner_vec); + col_builder.push_into((k, v, &time, d)); + } + let mut session = output.session(&new_cap); + while let Some(container) = col_builder.finish() { + session.give_container(container); + } + }); + } + }); + + stream.as_collection() +} + +/// Extract a `Collection<_, RecordedUpdates>` from a columnar `Arranged`. +/// +/// Cursors through each batch and pushes `(key, val, time, diff)` refs into +/// a `ValColBuilder`, which sorts and consolidates on flush. +pub fn as_recorded_updates( + arranged: differential_dataflow::operators::arrange::Arranged< + G, + differential_dataflow::operators::arrange::TraceAgent>, + >, +) -> differential_dataflow::Collection> +where + G: timely::dataflow::Scope, + U: layout::ColumnarUpdate, +{ + use timely::dataflow::operators::generic::Operator; + use timely::dataflow::channels::pact::Pipeline; + use differential_dataflow::trace::{BatchReader, Cursor}; + use differential_dataflow::AsCollection; + + arranged.stream + .unary::, _, _, _>(Pipeline, "AsRecordedUpdates", |_, _| { + move |input, output| { + input.for_each(|time, batches| { + let mut session = output.session_with_builder(&time); + for batch in batches.drain(..) { + let mut cursor = batch.cursor(); + while cursor.key_valid(&batch) { + while cursor.val_valid(&batch) { + let key = cursor.key(&batch); + let val = cursor.val(&batch); + cursor.map_times(&batch, |time, diff| { + session.give((key, val, time, diff)); + }); + cursor.step_val(&batch); + } + cursor.step_key(&batch); + } + } + }); + } + }) + .as_collection() +} diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs new file mode 100644 index 000000000..8404e7aa4 --- /dev/null +++ b/differential-dataflow/examples/columnar/main.rs @@ -0,0 +1,184 @@ +//! Columnar reachability example for differential dataflow. +//! +//! Demonstrates columnar-backed arrangements in an iterative scope, +//! exercising Enter, Leave, Negate, ResultsIn on RecordedUpdates, +//! and Push on Updates for the reduce builder path. + +mod columnar_support; + +use timely::container::{ContainerBuilder, PushInto}; +use timely::dataflow::InputHandle; +use timely::dataflow::ProbeHandle; + +use columnar_support::*; + +use mimalloc::MiMalloc; + +#[global_allocator] +static GLOBAL: MiMalloc = MiMalloc; + +fn main() { + + let nodes: u32 = std::env::args().nth(1).unwrap_or("100".into()).parse().unwrap(); + let edges: u32 = std::env::args().nth(2).unwrap_or("300".into()).parse().unwrap(); + + let timer = ::std::time::Instant::now(); + + timely::execute_from_args(std::env::args(), move |worker| { + + type EdgeUpdate = (u32, u32, u64, i64); + type NodeUpdate = (u32, (), u64, i64); + type EdgeBuilder = ValColBuilder; + type NodeBuilder = ValColBuilder; + + let mut edge_input = >::new_with_builder(); + let mut root_input = >::new_with_builder(); + let mut probe = ProbeHandle::new(); + + worker.dataflow::(|scope| { + use differential_dataflow::AsCollection; + use timely::dataflow::operators::Probe; + + let edges = edge_input.to_stream(scope).as_collection(); + let roots = root_input.to_stream(scope).as_collection(); + + reachability::reach(edges, roots) + .inner + .probe_with(&mut probe); + }); + + // Generate a small random graph. + let mut edge_builder = EdgeBuilder::default(); + let mut node_builder = NodeBuilder::default(); + + if worker.index() == 0 { + // Simple deterministic "random" edges. + let mut src: u32 = 0; + for _ in 0..edges { + let dst = (src.wrapping_mul(7).wrapping_add(13)) % nodes; + edge_builder.push_into((src, dst, 0u64, 1i64)); + src = (src + 1) % nodes; + } + // Root: node 0. + node_builder.push_into((0u32, (), 0u64, 1i64)); + } + + while let Some(container) = edge_builder.finish() { + edge_input.send_batch(container); + } + while let Some(container) = node_builder.finish() { + root_input.send_batch(container); + } + + edge_input.advance_to(1); + root_input.advance_to(1); + edge_input.flush(); + root_input.flush(); + + while probe.less_than(edge_input.time()) { + worker.step_or_park(None); + } + + println!("{:?}\treachability complete ({} nodes, {} edges)", timer.elapsed(), nodes, edges); + + }).unwrap(); + println!("{:?}\tshut down", timer.elapsed()); +} + +/// Reachability on a random directed graph using columnar containers. +/// +/// This module exercises the container traits needed for iterative columnar +/// computation: Enter, Leave, Negate, ResultsIn on RecordedUpdates, and +/// Push on Updates for the reduce builder path. +mod reachability { + + use timely::order::Product; + use timely::dataflow::Scope; + use differential_dataflow::Collection; + use differential_dataflow::AsCollection; + use differential_dataflow::operators::iterate::Variable; + use differential_dataflow::operators::arrange::arrangement::arrange_core; + use differential_dataflow::operators::join::join_traces; + + use crate::columnar_support::*; + + type Node = u32; + type Time = u64; + type Diff = i64; + type IterTime = Product; + + /// Compute the set of nodes reachable from `roots` along directed `edges`. + /// + /// Returns `(node, ())` for each reachable node. + pub fn reach>( + edges: Collection>, + roots: Collection>, + ) -> Collection> + { + let mut scope = edges.inner.scope(); + + scope.iterative::(|nested| { + let summary = Product::new(Time::default(), 1); + + let roots_inner = roots.enter(nested); + let (variable, reach) = Variable::new_from(roots_inner.clone(), summary); + let edges_inner = edges.enter(nested); + + // Arrange both collections into columnar spines for joining. + let edges_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; + let reach_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; + + let edges_arr = arrange_core::<_, _, + ValBatcher, + ValBuilder, + ValSpine, + >(edges_inner.inner, edges_pact, "Edges"); + + let reach_arr = arrange_core::<_, _, + ValBatcher, + ValBuilder, + ValSpine, + >(reach.inner, reach_pact, "Reach"); + + // join_traces with ValColBuilder: produces Stream<_, RecordedUpdates<...>>. + let proposed = + join_traces::<_, _, _, _, ValColBuilder<(Node, (), IterTime, Diff)>>( + edges_arr, + reach_arr, + |_src, dst, (), time, d1, d2, session| { + use differential_dataflow::difference::Multiply; + let dst: Node = *dst; + let diff: Diff = d1.clone().multiply(d2); + session.give::<(Node, (), IterTime, Diff)>((dst, (), time.clone(), diff)); + }, + ).as_collection(); + + // concat: both sides are now Collection<_, RecordedUpdates<...>>. + let combined = proposed.concat(roots_inner); + + // Arrange for reduce. + let combined_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; + let combined_arr = arrange_core::<_, _, + ValBatcher, + ValBuilder, + ValSpine, + >(combined.inner, combined_pact, "Combined"); + + // reduce_abelian on the columnar arrangement. + let result = combined_arr.reduce_abelian::<_, + ValBuilder, + ValSpine, + >("Distinct", |_node, _input, output| { + output.push(((), 1)); + }); + + // Extract RecordedUpdates from the Arranged's batch stream. + let result_col = as_recorded_updates::<_, (Node, (), IterTime, Diff)>(result); + + variable.set(result_col.clone()); + + // Leave the iterative scope. + result_col.leave() + }) + } +} diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 3d5d8c65c..2728de6e2 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -77,7 +77,8 @@ where // .reduce(|_, s, t| t.push((s[0].0.clone(), 1))) // }) - nodes.scope().iterative::(|scope| { + use timely::order::Product; + nodes.scope().scoped::,_,_>("Propagate", |scope| { use crate::operators::iterate::Variable; use crate::trace::implementations::{ValBuilder, ValSpine}; diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index f7bca525f..b730302e7 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -6,32 +6,11 @@ use std::hash::Hash; use timely::dataflow::*; use crate::{VecCollection, ExchangeData}; -use crate::operators::*; use crate::lattice::Lattice; use crate::difference::{Abelian, Multiply}; use super::propagate::propagate; -/// Iteratively removes nodes with no in-edges. -pub fn trim(graph: VecCollection) -> VecCollection -where - G: Scope, - N: ExchangeData + Hash, - R: ExchangeData + Abelian, - R: Multiply, - R: From, -{ - graph.clone().iterate(|scope, edges| { - // keep edges from active edge destinations. - let graph = graph.enter(&scope); - let active = - edges.map(|(_src,dst)| dst) - .threshold(|_,c| if c.is_zero() { R::from(0_i8) } else { R::from(1_i8) }); - - graph.semijoin(active) - }) -} - /// Returns the subset of edges in the same strongly connected component. pub fn strongly_connected(graph: VecCollection) -> VecCollection where @@ -41,10 +20,18 @@ where R: Multiply, R: From { - graph.clone().iterate(|scope, inner| { + use timely::order::Product; + graph.scope().scoped::,_,_>("StronglyConnected", |scope| { + // Bring in edges and transposed edges. let edges = graph.enter(&scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); - trim_edges(trim_edges(inner, edges), trans) + // Create a new variable that will be intra-scc edges. + use crate::operators::iterate::Variable; + let (variable, inner) = Variable::new_from(edges.clone(), Product::new(Default::default(), 1)); + + let result = trim_edges(trim_edges(inner, edges), trans); + variable.set(result.clone()); + result.leave() }) } @@ -57,16 +44,24 @@ where R: Multiply, R: From { - let nodes = edges.clone() - .map_in_place(|x| x.0 = x.1.clone()) - .consolidate(); + edges.inner.scope().region_named("TrimEdges", |region| { + let cycle = cycle.enter_region(region); + let edges = edges.enter_region(region); + + let nodes = edges.clone() + .map_in_place(|x| x.0 = x.1.clone()) + .consolidate(); - // NOTE: With a node -> int function, can be improved by: - // let labels = propagate_at(&cycle, &nodes, |x| *x as u64); - let labels = propagate(cycle, nodes); + // NOTE: With a node -> int function, can be improved by: + // let labels = propagate_at(&cycle, &nodes, |x| *x as u64); + let labels = propagate(cycle, nodes).arrange_by_key(); - edges.join_map(labels.clone(), |e1,e2,l1| (e2.clone(),(e1.clone(),l1.clone()))) - .join_map(labels, |e2,(e1,l1),l2| ((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))) - .filter(|(_,(l1,l2))| l1 == l2) - .map(|((x1,x2),_)| (x2,x1)) + edges.arrange_by_key() + .join_core(labels.clone(), |e1,e2,l1| [(e2.clone(),(e1.clone(),l1.clone()))]) + .arrange_by_key() + .join_core(labels, |e2,(e1,l1),l2| [((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))]) + .filter(|(_,(l1,l2))| l1 == l2) + .map(|((x1,x2),_)| (x2,x1)) + .leave_region() + }) } diff --git a/differential-dataflow/src/consolidation.rs b/differential-dataflow/src/consolidation.rs index 0d5271b9e..43d403d38 100644 --- a/differential-dataflow/src/consolidation.rs +++ b/differential-dataflow/src/consolidation.rs @@ -10,11 +10,11 @@ //! you need specific behavior, it may be best to defensively copy, paste, and maintain the //! specific behavior you require. -use std::cmp::Ordering; use std::collections::VecDeque; -use timely::container::{ContainerBuilder, DrainContainer, PushInto}; +use columnation::Columnation; +use timely::container::{ContainerBuilder, PushInto}; use crate::Data; -use crate::difference::{IsZero, Semigroup}; +use crate::difference::Semigroup; /// Sorts and consolidates `vec`. /// @@ -232,115 +232,63 @@ where } } -/// Layout of containers and their read items to be consolidated. +/// A container that can sort and consolidate its contents internally. /// -/// This trait specifies behavior to extract keys and diffs from container's read -/// items. Consolidation accumulates the diffs per key. +/// The container knows its own layout — how to sort its elements, how to +/// compare adjacent entries, and how to merge diffs. The caller provides +/// a `target` container to receive the consolidated output, allowing +/// reuse of allocations across calls. /// -/// The trait requires `Container` to have access to its `Item` GAT. -pub trait ConsolidateLayout: DrainContainer { - /// Key portion of data, essentially everything minus the diff - type Key<'a>: Eq where Self: 'a; - - /// GAT diff type. - type Diff<'a>; - - /// Owned diff type. - type DiffOwned: for<'a> Semigroup>; - - /// Converts a reference diff into an owned diff. - fn owned_diff(diff: Self::Diff<'_>) -> Self::DiffOwned; - - /// Deconstruct an item into key and diff. Must be cheap. - fn into_parts(item: Self::Item<'_>) -> (Self::Key<'_>, Self::Diff<'_>); - - /// Push an element to a compatible container. - /// - /// This function is odd to have, so let's explain why it exists. Ideally, the container - /// would accept a `(key, diff)` pair and we wouldn't need this function. However, we - /// might never be in a position where this is true: Vectors can push any `T`, which would - /// collide with a specific implementation for pushing tuples of mixes GATs and owned types. - /// - /// For this reason, we expose a function here that takes a GAT key and an owned diff, and - /// leave it to the implementation to "patch" a suitable item that can be pushed into `self`. - fn push_with_diff(&mut self, key: Self::Key<'_>, diff: Self::DiffOwned); - - /// Compare two items by key to sort containers. - fn cmp(item1: &Self::Item<'_>, item2: &Self::Item<'_>) -> Ordering; - - /// Returns the number of items in the container. +/// After the call, `target` contains the sorted, consolidated data and +/// `self` may be empty or in an unspecified state (implementations should +/// document this). +pub trait Consolidate { + /// The number of elements in the container. fn len(&self) -> usize; - - /// Clear the container. Afterwards, `len()` should return 0. + /// Clear the container. fn clear(&mut self); + /// Sort and consolidate `self` into `target`. + fn consolidate_into(&mut self, target: &mut Self); +} - /// Consolidate the supplied container. +impl Consolidate for Vec<(D, T, R)> { + fn len(&self) -> usize { Vec::len(self) } + fn clear(&mut self) { Vec::clear(self) } fn consolidate_into(&mut self, target: &mut Self) { - // Sort input data - let mut permutation = Vec::with_capacity(self.len()); - permutation.extend(self.drain()); - permutation.sort_by(|a, b| Self::cmp(a, b)); - - // Iterate over the data, accumulating diffs for like keys. - let mut iter = permutation.drain(..); - if let Some(item) = iter.next() { - - let (k, d) = Self::into_parts(item); - let mut prev_key = k; - let mut prev_diff = Self::owned_diff(d); - - for item in iter { - let (next_key, next_diff) = Self::into_parts(item); - if next_key == prev_key { - prev_diff.plus_equals(&next_diff); - } - else { - if !prev_diff.is_zero() { - target.push_with_diff(prev_key, prev_diff); - } - prev_key = next_key; - prev_diff = Self::owned_diff(next_diff); - } - } - - if !prev_diff.is_zero() { - target.push_with_diff(prev_key, prev_diff); - } - } + consolidate_updates(self); + std::mem::swap(self, target); } } -impl ConsolidateLayout for Vec<(D, T, R)> -where - D: Ord + Clone + 'static, - T: Ord + Clone + 'static, - R: Semigroup + Clone + 'static, -{ - type Key<'a> = (D, T) where Self: 'a; - type Diff<'a> = R where Self: 'a; - type DiffOwned = R; - - fn owned_diff(diff: Self::Diff<'_>) -> Self::DiffOwned { diff } - - fn into_parts((data, time, diff): Self::Item<'_>) -> (Self::Key<'_>, Self::Diff<'_>) { - ((data, time), diff) - } - - fn cmp<'a>(item1: &Self::Item<'_>, item2: &Self::Item<'_>) -> Ordering { - (&item1.0, &item1.1).cmp(&(&item2.0, &item2.1)) - } - - fn push_with_diff(&mut self, (data, time): Self::Key<'_>, diff: Self::DiffOwned) { - self.push((data, time, diff)); - } - - #[inline] fn len(&self) -> usize { Vec::len(self) } - #[inline] fn clear(&mut self) { Vec::clear(self) } - - /// Consolidate the supplied container. +impl Consolidate for crate::containers::TimelyStack<(D, T, R)> { + fn len(&self) -> usize { self[..].len() } + fn clear(&mut self) { crate::containers::TimelyStack::clear(self) } fn consolidate_into(&mut self, target: &mut Self) { - consolidate_updates(self); - std::mem::swap(self, target); + let len = self[..].len(); + let mut indices: Vec = (0..len).collect(); + indices.sort_unstable_by(|&i, &j| { + let (d1, t1, _) = &self[i]; + let (d2, t2, _) = &self[j]; + (d1, t1).cmp(&(d2, t2)) + }); + target.clear(); + let mut idx = 0; + while idx < indices.len() { + let (d, t, r) = &self[indices[idx]]; + let mut r_owned = r.clone(); + idx += 1; + while idx < indices.len() { + let (d2, t2, r2) = &self[indices[idx]]; + if d == d2 && t == t2 { + r_owned.plus_equals(r2); + idx += 1; + } else { break; } + } + if !r_owned.is_zero() { + target.copy_destructured(d, t, &r_owned); + } + } + self.clear(); } } diff --git a/differential-dataflow/src/difference.rs b/differential-dataflow/src/difference.rs index 9aab05384..401bf565f 100644 --- a/differential-dataflow/src/difference.rs +++ b/differential-dataflow/src/difference.rs @@ -164,7 +164,7 @@ mod present { /// The primary feature of this type is that it has zero size, which reduces the overhead /// of differential dataflow's representations for settings where collections either do /// not change, or for which records are only added (for example, derived facts in Datalog). - #[derive(Copy, Ord, PartialOrd, Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash)] + #[derive(Copy, Ord, PartialOrd, Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash, columnar::Columnar)] pub struct Present; impl super::Multiply for Present { @@ -181,6 +181,10 @@ mod present { impl super::Semigroup for Present { fn plus_equals(&mut self, _rhs: &Self) { } } + + impl columnation::Columnation for Present { + type InnerRegion = columnation::CopyRegion; + } } // Pair implementations. diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index 3fd612107..43a4987e8 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -258,7 +258,7 @@ impl InputSession { /// called, all buffers are flushed and timely dataflow is advised that some logical times are no longer possible. pub fn flush(&mut self) { self.handle.send_batch(&mut self.buffer); - if self.handle.epoch().less_than(&self.time) { + if self.handle.time().less_than(&self.time) { self.handle.advance_to(self.time.clone()); } } @@ -269,7 +269,7 @@ impl InputSession { /// the session is dropped or flushed. It is not correct to use this time as a basis for a computation's `step_while` /// method unless the session has just been flushed. pub fn advance_to(&mut self, time: T) { - assert!(self.handle.epoch().less_equal(&time)); + assert!(self.handle.time().less_equal(&time)); assert!(&self.time.less_equal(&time)); self.time = time; } diff --git a/differential-dataflow/src/lib.rs b/differential-dataflow/src/lib.rs index 615aa5ded..b5561cfb2 100644 --- a/differential-dataflow/src/lib.rs +++ b/differential-dataflow/src/lib.rs @@ -53,7 +53,7 @@ //! //! ```ignore //! loop { -//! let time = input.epoch(); +//! let time = input.time(); //! for round in time .. time + 100 { //! input.advance_to(round); //! input.insert((round % 13, round % 7)); diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 5aaf8bb33..69965e65b 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -30,7 +30,7 @@ use crate::{Data, VecCollection, AsCollection}; use crate::difference::Semigroup; use crate::lattice::Lattice; use crate::trace::{self, Trace, TraceReader, BatchReader, Batcher, Builder, Cursor}; -use crate::trace::implementations::merge_batcher::container::MergerChunk; +use crate::trace::implementations::merge_batcher::container::InternalMerge; use trace::wrappers::enter::{TraceEnter, BatchEnter,}; use trace::wrappers::enter_at::TraceEnter as TraceEnterAt; @@ -130,7 +130,29 @@ where } } - /// Flattens the stream into a `Collection`. + /// Extracts a collection of any container from the stream of batches. + /// + /// This method is like `self.stream.flat_map`, except that it produces containers + /// directly, rather than form a container of containers as `flat_map` would. + pub fn as_container(self, mut logic: L) -> crate::Collection + where + I: IntoIterator, + L: FnMut(Tr::Batch) -> I+'static, + { + self.stream.unary(Pipeline, "AsContainer", move |_,_| move |input, output| { + input.for_each(|time, data| { + let mut session = output.session(&time); + for wrapper in data.drain(..) { + for mut container in logic(wrapper) { + session.give_container(&mut container); + } + } + }); + }) + .as_collection() + } + + /// Flattens the stream into a `VecCollection`. /// /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than @@ -142,7 +164,7 @@ where self.flat_map_ref(move |key, val| Some(logic(key,val))) } - /// Flattens the stream into a `Collection`. + /// Flattens the stream into a `VecCollection`. /// /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than @@ -155,7 +177,7 @@ where self.flat_map_ref(move |key, val| [(Tr::owned_key(key), Tr::owned_val(val))]) } - /// Extracts elements from an arrangement as a collection. + /// Extracts elements from an arrangement as a `VecCollection`. /// /// The supplied logic may produce an iterator over output values, allowing either /// filtering or flat mapping as part of the extraction. @@ -167,7 +189,7 @@ where Self::flat_map_batches(self.stream, logic) } - /// Extracts elements from a stream of batches as a collection. + /// Extracts elements from a stream of batches as a `VecCollection`. /// /// The supplied logic may produce an iterator over output values, allowing either /// filtering or flat mapping as part of the extraction. @@ -259,7 +281,7 @@ where Time=T1::Time, Diff: Abelian, >+'static, - Bu: Builder>, + Bu: Builder>, L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, { self.reduce_core::<_,Bu,T2>(name, move |key, input, output, change| { @@ -281,7 +303,7 @@ where ValOwn: Data, Time=T1::Time, >+'static, - Bu: Builder>, + Bu: Builder>, L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, { use crate::operators::reduce::reduce_trace; diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index f0302989f..fe3622d74 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -20,7 +20,7 @@ use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; use crate::trace::cursor::CursorList; use crate::trace::implementations::containers::BatchContainer; -use crate::trace::implementations::merge_batcher::container::MergerChunk; +use crate::trace::implementations::merge_batcher::container::InternalMerge; use crate::trace::TraceReader; /// A key-wise reduction of values in an input trace. @@ -31,7 +31,7 @@ where G: Scope, T1: TraceReader + Clone + 'static, T2: for<'a> Trace=T1::Key<'a>, KeyOwn=T1::KeyOwn, ValOwn: Data, Time=T1::Time> + 'static, - Bu: Builder>, + Bu: Builder>, L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn,T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, { let mut result_trace = None; diff --git a/differential-dataflow/src/trace/implementations/chunker.rs b/differential-dataflow/src/trace/implementations/chunker.rs index 71d7aeb41..93a9e516f 100644 --- a/differential-dataflow/src/trace/implementations/chunker.rs +++ b/differential-dataflow/src/trace/implementations/chunker.rs @@ -7,124 +7,15 @@ use timely::Container; use timely::container::{ContainerBuilder, DrainContainer, PushInto, SizableContainer}; use crate::containers::TimelyStack; -use crate::consolidation::{consolidate_updates, ConsolidateLayout}; +use crate::consolidation::{consolidate_updates, Consolidate}; use crate::difference::Semigroup; -/// Chunk a stream of vectors into chains of vectors. -pub struct VecChunker { - pending: Vec, - ready: VecDeque>, - empty: Option>, -} - -impl Default for VecChunker { - fn default() -> Self { - Self { - pending: Vec::default(), - ready: VecDeque::default(), - empty: None, - } - } -} - -impl VecChunker<((K, V), T, R)> -where - K: Ord, - V: Ord, - T: Ord, - R: Semigroup, -{ - const BUFFER_SIZE_BYTES: usize = 8 << 10; - fn chunk_capacity() -> usize { - let size = ::std::mem::size_of::<((K, V), T, R)>(); - if size == 0 { - Self::BUFFER_SIZE_BYTES - } else if size <= Self::BUFFER_SIZE_BYTES { - Self::BUFFER_SIZE_BYTES / size - } else { - 1 - } - } - - /// Form chunks out of pending data, if needed. This function is meant to be applied to - /// potentially full buffers, and ensures that if the buffer was full when called it is at most - /// half full when the function returns. - /// - /// `form_chunk` does the following: - /// * If pending is full, consolidate. - /// * If after consolidation it's more than half full, peel off chunks, - /// leaving behind any partial chunk in pending. - fn form_chunk(&mut self) { - consolidate_updates(&mut self.pending); - if self.pending.len() >= Self::chunk_capacity() { - while self.pending.len() > Self::chunk_capacity() { - let mut chunk = Vec::with_capacity(Self::chunk_capacity()); - chunk.extend(self.pending.drain(..chunk.capacity())); - self.ready.push_back(chunk); - } - } - } -} - -impl<'a, K, V, T, R> PushInto<&'a mut Vec<((K, V), T, R)>> for VecChunker<((K, V), T, R)> -where - K: Ord + Clone, - V: Ord + Clone, - T: Ord + Clone, - R: Semigroup + Clone, -{ - fn push_into(&mut self, container: &'a mut Vec<((K, V), T, R)>) { - // Ensure `self.pending` has the desired capacity. We should never have a larger capacity - // because we don't write more than capacity elements into the buffer. - // Important: Consolidation requires `pending` to have twice the chunk capacity to - // amortize its cost. Otherwise, it risks to do quadratic work. - if self.pending.capacity() < Self::chunk_capacity() * 2 { - self.pending.reserve(Self::chunk_capacity() * 2 - self.pending.len()); - } - - let mut drain = container.drain(..).peekable(); - while drain.peek().is_some() { - self.pending.extend((&mut drain).take(self.pending.capacity() - self.pending.len())); - if self.pending.len() == self.pending.capacity() { - self.form_chunk(); - } - } - } -} - -impl ContainerBuilder for VecChunker<((K, V), T, R)> -where - K: Ord + Clone + 'static, - V: Ord + Clone + 'static, - T: Ord + Clone + 'static, - R: Semigroup + Clone + 'static, -{ - type Container = Vec<((K, V), T, R)>; - - fn extract(&mut self) -> Option<&mut Self::Container> { - if let Some(ready) = self.ready.pop_front() { - self.empty = Some(ready); - self.empty.as_mut() - } else { - None - } - } - - fn finish(&mut self) -> Option<&mut Self::Container> { - if !self.pending.is_empty() { - consolidate_updates(&mut self.pending); - while !self.pending.is_empty() { - let mut chunk = Vec::with_capacity(Self::chunk_capacity()); - chunk.extend(self.pending.drain(..std::cmp::min(self.pending.len(), chunk.capacity()))); - self.ready.push_back(chunk); - } - } - self.empty = self.ready.pop_front(); - self.empty.as_mut() - } -} - -/// Chunk a stream of vectors into chains of vectors. +/// Chunk a stream of vectors into chains of columnation stacks. +/// +/// This chunker accumulates into a `Vec` (not a `TimelyStack`) for efficient +/// in-place sorting and consolidation, then copies the consolidated results +/// into `TimelyStack` chunks. This avoids the cost of sorting through +/// columnation indirection. pub struct ColumnationChunker { pending: Vec, ready: VecDeque>, @@ -159,14 +50,6 @@ where } } - /// Form chunks out of pending data, if needed. This function is meant to be applied to - /// potentially full buffers, and ensures that if the buffer was full when called it is at most - /// half full when the function returns. - /// - /// `form_chunk` does the following: - /// * If pending is full, consolidate. - /// * If after consolidation it's more than half full, peel off chunks, - /// leaving behind any partial chunk in pending. fn form_chunk(&mut self) { consolidate_updates(&mut self.pending); if self.pending.len() >= Self::chunk_capacity() { @@ -188,8 +71,6 @@ where R: Columnation + Semigroup + Clone, { fn push_into(&mut self, container: &'a mut Vec<(D, T, R)>) { - // Ensure `self.pending` has the desired capacity. We should never have a larger capacity - // because we don't write more than capacity elements into the buffer. if self.pending.capacity() < Self::chunk_capacity() * 2 { self.pending.reserve(Self::chunk_capacity() * 2 - self.pending.len()); } @@ -257,7 +138,7 @@ where Input: DrainContainer, Output: Default + SizableContainer - + ConsolidateLayout + + Consolidate + PushInto>, { fn push_into(&mut self, container: &'a mut Input) { @@ -283,7 +164,7 @@ where impl ContainerBuilder for ContainerChunker where - Output: SizableContainer + ConsolidateLayout + Container, + Output: SizableContainer + Consolidate + Container, { type Container = Output; diff --git a/differential-dataflow/src/trace/implementations/merge_batcher.rs b/differential-dataflow/src/trace/implementations/merge_batcher.rs index 5c932af9f..a1b818c6c 100644 --- a/differential-dataflow/src/trace/implementations/merge_batcher.rs +++ b/differential-dataflow/src/trace/implementations/merge_batcher.rs @@ -219,93 +219,80 @@ pub trait Merger: Default { fn account(chunk: &Self::Chunk) -> (usize, usize, usize, usize); } -pub use container::{VecMerger, ColMerger}; +pub use container::InternalMerger; pub mod container { - //! A general purpose `Merger` implementation for arbitrary containers. + //! Merger implementations for the merge batcher. //! - //! The implementation requires implementations of two traits, `ContainerQueue` and `MergerChunk`. - //! The `ContainerQueue` trait is meant to wrap a container and provide iterable access to it, as - //! well as the ability to return the container when iteration is complete. - //! The `MergerChunk` trait is meant to be implemented by containers, and it explains how container - //! items should be interpreted with respect to times, and with respect to differences. - //! These two traits exist instead of a stack of constraints on the structure of the associated items - //! of the containers, allowing them to perform their functions without destructuring their guts. - //! - //! Standard implementations exist in the `vec`, `columnation`, and `flat_container` modules. + //! The `InternalMerge` trait allows containers to merge sorted, consolidated + //! data using internal iteration. The `InternalMerger` type implements the + //! `Merger` trait using `InternalMerge`, and is the standard merger for all + //! container types. - use std::cmp::Ordering; use std::marker::PhantomData; - use timely::container::{PushInto, SizableContainer}; + use timely::container::SizableContainer; use timely::progress::frontier::{Antichain, AntichainRef}; use timely::{Accountable, PartialOrder}; - use timely::container::DrainContainer; use crate::trace::implementations::merge_batcher::Merger; - /// An abstraction for a container that can be iterated over, and conclude by returning itself. - pub trait ContainerQueue { - /// Returns either the next item in the container, or the container itself. - fn next_or_alloc(&mut self) -> Result, C>; - /// Indicates whether `next_or_alloc` will return `Ok`, and whether `peek` will return `Some`. - fn is_empty(&self) -> bool; - /// Compare the heads of two queues, where empty queues come last. - fn cmp_heads(&self, other: &Self) -> std::cmp::Ordering; - /// Create a new queue from an existing container. - fn from(container: C) -> Self; - } - - /// Behavior to dissect items of chunks in the merge batcher - pub trait MergerChunk : Accountable + DrainContainer + SizableContainer + Default { - /// An owned time type. - /// - /// This type is provided so that users can maintain antichains of something, in order to track - /// the forward movement of time and extract intervals from chains of updates. + /// A container that supports the operations needed by the merge batcher: + /// merging sorted chains and extracting updates by time. + pub trait InternalMerge: Accountable + SizableContainer + Default { + /// The owned time type, for maintaining antichains. type TimeOwned; - /// The owned diff type. - /// - /// This type is provided so that users can provide an owned instance to the `push_and_add` method, - /// to act as a scratch space when the type is substantial and could otherwise require allocations. - type DiffOwned: Default; - /// Relates a borrowed time to antichains of owned times. - /// - /// If `upper` is less or equal to `time`, the method returns `true` and ensures that `frontier` reflects `time`. - fn time_kept(time1: &Self::Item<'_>, upper: &AntichainRef, frontier: &mut Antichain) -> bool; + /// The number of items in this container. + fn len(&self) -> usize; - /// Push an entry that adds together two diffs. - /// - /// This is only called when two items are deemed mergeable by the container queue. - /// If the two diffs added together is zero do not push anything. - fn push_and_add<'a>(&mut self, item1: Self::Item<'a>, item2: Self::Item<'a>, stash: &mut Self::DiffOwned); + /// Clear the container for reuse. + fn clear(&mut self); /// Account the allocations behind the chunk. - // TODO: Find a more universal home for this: `Container`? fn account(&self) -> (usize, usize, usize, usize) { let (size, capacity, allocations) = (0, 0, 0); (usize::try_from(self.record_count()).unwrap(), size, capacity, allocations) } - /// Clear the chunk, to be reused. - fn clear(&mut self); + /// Merge items from sorted inputs into `self`, advancing positions. + /// Merges until `self` is at capacity or all inputs are exhausted. + /// + /// Dispatches based on the number of inputs: + /// - **0**: no-op + /// - **1**: bulk copy (may swap the input into `self`) + /// - **2**: merge two sorted streams + fn merge_from( + &mut self, + others: &mut [Self], + positions: &mut [usize], + ); + + /// Extract updates from `self` into `ship` (times not beyond `upper`) + /// and `keep` (times beyond `upper`), updating `frontier` with kept times. + fn extract( + &mut self, + upper: AntichainRef, + frontier: &mut Antichain, + keep: &mut Self, + ship: &mut Self, + ); } - /// A merger for arbitrary containers. - /// - /// `MC` is a `Container` that implements [`MergerChunk`]. - /// `CQ` is a [`ContainerQueue`] supporting `MC`. - pub struct ContainerMerger { - _marker: PhantomData<(MC, CQ)>, + /// A `Merger` using internal iteration for `Vec` containers. + pub type VecInternalMerger = InternalMerger>; + /// A `Merger` using internal iteration for `TimelyStack` containers. + pub type ColInternalMerger = InternalMerger>; + + /// A merger that uses internal iteration via [`InternalMerge`]. + pub struct InternalMerger { + _marker: PhantomData, } - impl Default for ContainerMerger { - fn default() -> Self { - Self { _marker: PhantomData, } - } + impl Default for InternalMerger { + fn default() -> Self { Self { _marker: PhantomData } } } - impl ContainerMerger { - /// Helper to get pre-sized vector from the stash. + impl InternalMerger where MC: InternalMerge { #[inline] fn empty(&self, stash: &mut Vec) -> MC { stash.pop().unwrap_or_else(|| { @@ -314,97 +301,87 @@ pub mod container { container }) } - /// Helper to return a chunk to the stash. #[inline] fn recycle(&self, mut chunk: MC, stash: &mut Vec) { - // TODO: Should we only retain correctly sized containers? chunk.clear(); stash.push(chunk); } + /// Drain remaining items from one side into `result`/`output`. + fn drain_side( + &self, + head: &mut MC, + pos: &mut usize, + list: &mut std::vec::IntoIter, + result: &mut MC, + output: &mut Vec, + stash: &mut Vec, + ) { + while *pos < head.len() { + result.merge_from( + std::slice::from_mut(head), + std::slice::from_mut(pos), + ); + if *pos >= head.len() { + let old = std::mem::replace(head, list.next().unwrap_or_default()); + self.recycle(old, stash); + *pos = 0; + } + if result.at_capacity() { + output.push(std::mem::take(result)); + *result = self.empty(stash); + } + } + } } - impl Merger for ContainerMerger + impl Merger for InternalMerger where - for<'a> MC: MergerChunk + Clone + PushInto<::Item<'a>> + 'static, - CQ: ContainerQueue, + MC: InternalMerge + 'static, { type Time = MC::TimeOwned; type Chunk = MC; - // TODO: Consider integrating with `ConsolidateLayout`. - fn merge(&mut self, list1: Vec, list2: Vec, output: &mut Vec, stash: &mut Vec) { + fn merge(&mut self, list1: Vec, list2: Vec, output: &mut Vec, stash: &mut Vec) { let mut list1 = list1.into_iter(); let mut list2 = list2.into_iter(); - let mut head1 = CQ::from(list1.next().unwrap_or_default()); - let mut head2 = CQ::from(list2.next().unwrap_or_default()); + let mut heads = [list1.next().unwrap_or_default(), list2.next().unwrap_or_default()]; + let mut positions = [0usize, 0usize]; let mut result = self.empty(stash); - let mut diff_owned = Default::default(); - - // while we have valid data in each input, merge. - while !head1.is_empty() && !head2.is_empty() { - while !result.at_capacity() && !head1.is_empty() && !head2.is_empty() { - let cmp = head1.cmp_heads(&head2); - // TODO: The following less/greater branches could plausibly be a good moment for - // `copy_range`, on account of runs of records that might benefit more from a - // `memcpy`. - match cmp { - Ordering::Less => { - result.push_into(head1.next_or_alloc().ok().unwrap()); - } - Ordering::Greater => { - result.push_into(head2.next_or_alloc().ok().unwrap()); - } - Ordering::Equal => { - let item1 = head1.next_or_alloc().ok().unwrap(); - let item2 = head2.next_or_alloc().ok().unwrap(); - result.push_and_add(item1, item2, &mut diff_owned); - } - } - } - - if result.at_capacity() { - output.push_into(result); - result = self.empty(stash); - } + // Main merge loop: both sides have data. + while positions[0] < heads[0].len() && positions[1] < heads[1].len() { + result.merge_from(&mut heads, &mut positions); - if head1.is_empty() { - self.recycle(head1.next_or_alloc().err().unwrap(), stash); - head1 = CQ::from(list1.next().unwrap_or_default()); + if positions[0] >= heads[0].len() { + let old = std::mem::replace(&mut heads[0], list1.next().unwrap_or_default()); + self.recycle(old, stash); + positions[0] = 0; } - if head2.is_empty() { - self.recycle(head2.next_or_alloc().err().unwrap(), stash); - head2 = CQ::from(list2.next().unwrap_or_default()); + if positions[1] >= heads[1].len() { + let old = std::mem::replace(&mut heads[1], list2.next().unwrap_or_default()); + self.recycle(old, stash); + positions[1] = 0; } - } - - // TODO: recycle `head1` rather than discarding. - while let Ok(next) = head1.next_or_alloc() { - result.push_into(next); if result.at_capacity() { - output.push_into(result); + output.push(std::mem::take(&mut result)); result = self.empty(stash); } } + + // Drain remaining from side 0. + self.drain_side(&mut heads[0], &mut positions[0], &mut list1, &mut result, output, stash); if !result.is_empty() { - output.push_into(result); + output.push(std::mem::take(&mut result)); result = self.empty(stash); } output.extend(list1); - // TODO: recycle `head2` rather than discarding. - while let Ok(next) = head2.next_or_alloc() { - result.push_into(next); - if result.at_capacity() { - output.push(result); - result = self.empty(stash); - } - } + // Drain remaining from side 1. + self.drain_side(&mut heads[1], &mut positions[1], &mut list2, &mut result, output, stash); if !result.is_empty() { - output.push_into(result); - // result = self.empty(stash); + output.push(std::mem::take(&mut result)); } output.extend(list2); } @@ -414,7 +391,7 @@ pub mod container { merged: Vec, upper: AntichainRef, frontier: &mut Antichain, - readied: &mut Vec, + ship: &mut Vec, kept: &mut Vec, stash: &mut Vec, ) { @@ -422,174 +399,137 @@ pub mod container { let mut ready = self.empty(stash); for mut buffer in merged { - for item in buffer.drain() { - if MC::time_kept(&item, &upper, frontier) { - if keep.at_capacity() && !keep.is_empty() { - kept.push(keep); - keep = self.empty(stash); - } - keep.push_into(item); - } else { - if ready.at_capacity() && !ready.is_empty() { - readied.push(ready); - ready = self.empty(stash); - } - ready.push_into(item); - } - } - // Recycling buffer. + buffer.extract(upper, frontier, &mut keep, &mut ready); self.recycle(buffer, stash); + if keep.at_capacity() { + kept.push(std::mem::take(&mut keep)); + keep = self.empty(stash); + } + if ready.at_capacity() { + ship.push(std::mem::take(&mut ready)); + ready = self.empty(stash); + } } - // Finish the kept data. if !keep.is_empty() { kept.push(keep); } if !ready.is_empty() { - readied.push(ready); + ship.push(ready); } } - /// Account the allocations behind the chunk. fn account(chunk: &Self::Chunk) -> (usize, usize, usize, usize) { chunk.account() } } - pub use vec::VecMerger; - /// Implementations of `ContainerQueue` and `MergerChunk` for `Vec` containers. - pub mod vec { - - use std::collections::VecDeque; - use timely::progress::{Antichain, frontier::AntichainRef}; + /// Implementation of `InternalMerge` for `Vec<(D, T, R)>`. + pub mod vec_internal { + use std::cmp::Ordering; + use timely::PartialOrder; + use timely::container::SizableContainer; + use timely::progress::frontier::{Antichain, AntichainRef}; use crate::difference::Semigroup; - use super::{ContainerQueue, MergerChunk}; - - /// A `Merger` implementation backed by vector containers. - pub type VecMerger = super::ContainerMerger, std::collections::VecDeque<(D, T, R)>>; - - impl ContainerQueue> for VecDeque<(D, T, R)> { - fn next_or_alloc(&mut self) -> Result<(D, T, R), Vec<(D, T, R)>> { - if self.is_empty() { - Err(Vec::from(std::mem::take(self))) - } - else { - Ok(self.pop_front().unwrap()) - } - } - fn is_empty(&self) -> bool { - self.is_empty() - } - fn cmp_heads(&self, other: &Self) -> std::cmp::Ordering { - let (data1, time1, _) = self.front().unwrap(); - let (data2, time2, _) = other.front().unwrap(); - (data1, time1).cmp(&(data2, time2)) - } - fn from(list: Vec<(D, T, R)>) -> Self { - >::from(list) - } - } + use super::InternalMerge; - impl MergerChunk for Vec<(D, T, R)> { + impl InternalMerge for Vec<(D, T, R)> { type TimeOwned = T; - type DiffOwned = (); - fn time_kept((_, time, _): &Self::Item<'_>, upper: &AntichainRef, frontier: &mut Antichain) -> bool { - if upper.less_equal(time) { - frontier.insert_with(&time, |time| time.clone()); - true + fn len(&self) -> usize { Vec::len(self) } + fn clear(&mut self) { Vec::clear(self) } + + fn merge_from( + &mut self, + others: &mut [Self], + positions: &mut [usize], + ) { + match others.len() { + 0 => {}, + 1 => { + let other = &mut others[0]; + let pos = &mut positions[0]; + if self.is_empty() && *pos == 0 { + std::mem::swap(self, other); + return; + } + self.extend_from_slice(&other[*pos ..]); + *pos = other.len(); + }, + 2 => { + let (left, right) = others.split_at_mut(1); + let other1 = &mut left[0]; + let other2 = &mut right[0]; + + while positions[0] < other1.len() && positions[1] < other2.len() && !self.at_capacity() { + let (d1, t1, _) = &other1[positions[0]]; + let (d2, t2, _) = &other2[positions[1]]; + match (d1, t1).cmp(&(d2, t2)) { + Ordering::Less => { + self.push(other1[positions[0]].clone()); + positions[0] += 1; + } + Ordering::Greater => { + self.push(other2[positions[1]].clone()); + positions[1] += 1; + } + Ordering::Equal => { + let (d, t, mut r1) = other1[positions[0]].clone(); + let (_, _, ref r2) = other2[positions[1]]; + r1.plus_equals(r2); + if !r1.is_zero() { + self.push((d, t, r1)); + } + positions[0] += 1; + positions[1] += 1; + } + } + } + }, + n => unimplemented!("{n}-way merge not yet supported"), } - else { false } } - fn push_and_add<'a>(&mut self, item1: Self::Item<'a>, item2: Self::Item<'a>, _stash: &mut Self::DiffOwned) { - let (data, time, mut diff1) = item1; - let (_data, _time, diff2) = item2; - diff1.plus_equals(&diff2); - if !diff1.is_zero() { - self.push((data, time, diff1)); + + fn extract( + &mut self, + upper: AntichainRef, + frontier: &mut Antichain, + keep: &mut Self, + ship: &mut Self, + ) { + for (data, time, diff) in self.drain(..) { + if upper.less_equal(&time) { + frontier.insert_with(&time, |time| time.clone()); + keep.push((data, time, diff)); + } else { + ship.push((data, time, diff)); + } } } - fn account(&self) -> (usize, usize, usize, usize) { - let (size, capacity, allocations) = (0, 0, 0); - (self.len(), size, capacity, allocations) - } - #[inline] fn clear(&mut self) { Vec::clear(self) } } } - pub use columnation::ColMerger; - /// Implementations of `ContainerQueue` and `MergerChunk` for `TimelyStack` containers (columnation). - pub mod columnation { - - use timely::progress::{Antichain, frontier::AntichainRef}; + /// Implementation of `InternalMerge` for `TimelyStack<(D, T, R)>`. + pub mod columnation_internal { + use std::cmp::Ordering; use columnation::Columnation; - + use timely::PartialOrder; + use timely::container::SizableContainer; + use timely::progress::frontier::{Antichain, AntichainRef}; use crate::containers::TimelyStack; use crate::difference::Semigroup; - - use super::{ContainerQueue, MergerChunk}; - - /// A `Merger` implementation backed by `TimelyStack` containers (columnation). - pub type ColMerger = super::ContainerMerger,TimelyStackQueue<(D, T, R)>>; - - /// TODO - pub struct TimelyStackQueue { - list: TimelyStack, - head: usize, - } - - impl ContainerQueue> for TimelyStackQueue<(D, T, R)> { - fn next_or_alloc(&mut self) -> Result<&(D, T, R), TimelyStack<(D, T, R)>> { - if self.is_empty() { - Err(std::mem::take(&mut self.list)) - } - else { - Ok(self.pop()) - } - } - fn is_empty(&self) -> bool { - self.head == self.list[..].len() - } - fn cmp_heads(&self, other: &Self) -> std::cmp::Ordering { - let (data1, time1, _) = self.peek(); - let (data2, time2, _) = other.peek(); - (data1, time1).cmp(&(data2, time2)) - } - fn from(list: TimelyStack<(D, T, R)>) -> Self { - TimelyStackQueue { list, head: 0 } - } - } - - impl TimelyStackQueue { - fn pop(&mut self) -> &T { - self.head += 1; - &self.list[self.head - 1] - } - - fn peek(&self) -> &T { - &self.list[self.head] - } - } - - impl MergerChunk for TimelyStack<(D, T, R)> { + use super::InternalMerge; + + impl InternalMerge for TimelyStack<(D, T, R)> + where + D: Ord + Columnation + Clone + 'static, + T: Ord + Columnation + Clone + PartialOrder + 'static, + R: Default + Semigroup + Columnation + Clone + 'static, + { type TimeOwned = T; - type DiffOwned = R; - fn time_kept((_, time, _): &Self::Item<'_>, upper: &AntichainRef, frontier: &mut Antichain) -> bool { - if upper.less_equal(time) { - frontier.insert_with(&time, |time| time.clone()); - true - } - else { false } - } - fn push_and_add<'a>(&mut self, item1: Self::Item<'a>, item2: Self::Item<'a>, stash: &mut Self::DiffOwned) { - let (data, time, diff1) = item1; - let (_data, _time, diff2) = item2; - stash.clone_from(diff1); - stash.plus_equals(&diff2); - if !stash.is_zero() { - self.copy_destructured(data, time, stash); - } - } + fn len(&self) -> usize { self[..].len() } + fn clear(&mut self) { TimelyStack::clear(self) } + fn account(&self) -> (usize, usize, usize, usize) { let (mut size, mut capacity, mut allocations) = (0, 0, 0); let cb = |siz, cap| { @@ -600,7 +540,80 @@ pub mod container { self.heap_size(cb); (self.len(), size, capacity, allocations) } - #[inline] fn clear(&mut self) { TimelyStack::clear(self) } + + fn merge_from( + &mut self, + others: &mut [Self], + positions: &mut [usize], + ) { + match others.len() { + 0 => {}, + 1 => { + let other = &mut others[0]; + let pos = &mut positions[0]; + if self[..].is_empty() && *pos == 0 { + std::mem::swap(self, other); + return; + } + for i in *pos .. other[..].len() { + self.copy(&other[i]); + } + *pos = other[..].len(); + }, + 2 => { + let (left, right) = others.split_at_mut(1); + let other1 = &left[0]; + let other2 = &right[0]; + + let mut stash = R::default(); + + while positions[0] < other1[..].len() && positions[1] < other2[..].len() && !self.at_capacity() { + let (d1, t1, _) = &other1[positions[0]]; + let (d2, t2, _) = &other2[positions[1]]; + match (d1, t1).cmp(&(d2, t2)) { + Ordering::Less => { + self.copy(&other1[positions[0]]); + positions[0] += 1; + } + Ordering::Greater => { + self.copy(&other2[positions[1]]); + positions[1] += 1; + } + Ordering::Equal => { + let (_, _, r1) = &other1[positions[0]]; + let (_, _, r2) = &other2[positions[1]]; + stash.clone_from(r1); + stash.plus_equals(r2); + if !stash.is_zero() { + let (d, t, _) = &other1[positions[0]]; + self.copy_destructured(d, t, &stash); + } + positions[0] += 1; + positions[1] += 1; + } + } + } + }, + n => unimplemented!("{n}-way merge not yet supported"), + } + } + + fn extract( + &mut self, + upper: AntichainRef, + frontier: &mut Antichain, + keep: &mut Self, + ship: &mut Self, + ) { + for (data, time, diff) in self.iter() { + if upper.less_equal(time) { + frontier.insert_with(time, |time| time.clone()); + keep.copy_destructured(data, time, diff); + } else { + ship.copy_destructured(data, time, diff); + } + } + } } } } diff --git a/differential-dataflow/src/trace/implementations/ord_neu.rs b/differential-dataflow/src/trace/implementations/ord_neu.rs index b6f9a198e..14bc8a2db 100644 --- a/differential-dataflow/src/trace/implementations/ord_neu.rs +++ b/differential-dataflow/src/trace/implementations/ord_neu.rs @@ -11,9 +11,10 @@ use std::rc::Rc; use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, VecChunker}; +use crate::trace::implementations::chunker::{ColumnationChunker, ContainerChunker}; use crate::trace::implementations::spine_fueled::Spine; -use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger, ColMerger}; +use crate::trace::implementations::merge_batcher::MergeBatcher; +use crate::trace::implementations::merge_batcher::container::{VecInternalMerger, ColInternalMerger}; use crate::trace::rc_blanket_impls::RcBuilder; use super::{Layout, Vector, TStack}; @@ -24,7 +25,7 @@ pub use self::key_batch::{OrdKeyBatch, OrdKeyBuilder}; /// A trace implementation using a spine of ordered lists. pub type OrdValSpine = Spine>>>; /// A batcher using ordered lists. -pub type OrdValBatcher = MergeBatcher, VecChunker<((K,V),T,R)>, VecMerger<(K, V), T, R>>; +pub type OrdValBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, V), T, R>>; /// A builder using ordered lists. pub type RcOrdValBuilder = RcBuilder, Vec<((K,V),T,R)>>>; @@ -34,14 +35,14 @@ pub type RcOrdValBuilder = RcBuilder = Spine>>>; /// A batcher for columnar storage. -pub type ColValBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColMerger<(K,V),T,R>>; +pub type ColValBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColInternalMerger<(K,V),T,R>>; /// A builder for columnar storage. pub type ColValBuilder = RcBuilder, TimelyStack<((K,V),T,R)>>>; /// A trace implementation using a spine of ordered lists. pub type OrdKeySpine = Spine>>>; /// A batcher for ordered lists. -pub type OrdKeyBatcher = MergeBatcher, VecChunker<((K,()),T,R)>, VecMerger<(K, ()), T, R>>; +pub type OrdKeyBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, ()), T, R>>; /// A builder for ordered lists. pub type RcOrdKeyBuilder = RcBuilder, Vec<((K,()),T,R)>>>; @@ -51,7 +52,7 @@ pub type RcOrdKeyBuilder = RcBuilder /// A trace implementation backed by columnar storage. pub type ColKeySpine = Spine>>>; /// A batcher for columnar storage -pub type ColKeyBatcher = MergeBatcher, ColumnationChunker<((K,()),T,R)>, ColMerger<(K,()),T,R>>; +pub type ColKeyBatcher = MergeBatcher, ColumnationChunker<((K,()),T,R)>, ColInternalMerger<(K,()),T,R>>; /// A builder for columnar storage pub type ColKeyBuilder = RcBuilder, TimelyStack<((K,()),T,R)>>>; diff --git a/differential-dataflow/src/trace/implementations/rhh.rs b/differential-dataflow/src/trace/implementations/rhh.rs index 71d20b69d..bf2f252a9 100644 --- a/differential-dataflow/src/trace/implementations/rhh.rs +++ b/differential-dataflow/src/trace/implementations/rhh.rs @@ -12,8 +12,9 @@ use serde::{Deserialize, Serialize}; use crate::Hashable; use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, VecChunker}; -use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger, ColMerger}; +use crate::trace::implementations::chunker::{ColumnationChunker, ContainerChunker}; +use crate::trace::implementations::merge_batcher::MergeBatcher; +use crate::trace::implementations::merge_batcher::container::{VecInternalMerger, ColInternalMerger}; use crate::trace::implementations::spine_fueled::Spine; use crate::trace::rc_blanket_impls::RcBuilder; @@ -24,7 +25,7 @@ use self::val_batch::{RhhValBatch, RhhValBuilder}; /// A trace implementation using a spine of ordered lists. pub type VecSpine = Spine>>>; /// A batcher for ordered lists. -pub type VecBatcher = MergeBatcher, VecChunker<((K,V),T,R)>, VecMerger<(K, V), T, R>>; +pub type VecBatcher = MergeBatcher, ContainerChunker>, VecInternalMerger<(K, V), T, R>>; /// A builder for ordered lists. pub type VecBuilder = RcBuilder, Vec<((K,V),T,R)>>>; @@ -34,7 +35,7 @@ pub type VecBuilder = RcBuilder, Vec< /// A trace implementation backed by columnar storage. pub type ColSpine = Spine>>>; /// A batcher for columnar storage. -pub type ColBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColMerger<(K,V),T,R>>; +pub type ColBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColInternalMerger<(K,V),T,R>>; /// A builder for columnar storage. pub type ColBuilder = RcBuilder, TimelyStack<((K,V),T,R)>>>;