mz_compute/render/continual_task.rs
1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! A continual task presents as something like a `TRIGGER`: it watches some
11//! _input_ and whenever it changes at time `T`, executes a SQL txn, writing to
12//! some _output_ at the same time `T`. It can also read anything in materialize
13//! as a _reference_, most notably including the output.
14//!
15//! Only reacting to new inputs (and not the full history) makes a CT's
16//! rehydration time independent of the size of the inputs (NB this is not true
17//! for references), enabling things like writing UPSERT on top of an
18//! append-only shard in SQL (ignore the obvious bug with my upsert impl):
19//!
20//! ```sql
21//! CREATE CONTINUAL TASK upsert (key INT, val INT) ON INPUT append_only AS (
22//! DELETE FROM upsert WHERE key IN (SELECT key FROM append_only);
23//! INSERT INTO upsert SELECT key, max(val) FROM append_only GROUP BY key;
24//! )
25//! ```
26//!
27//! Unlike a materialized view, the continual task does not update outputs if
28//! references later change. This enables things like auditing:
29//!
30//! ```sql
31//! CREATE CONTINUAL TASK audit_log (count INT8) ON INPUT anomalies AS (
32//! INSERT INTO audit_log SELECT * FROM anomalies;
33//! )
34//! ```
35//!
36//! Rough implementation overview:
37//! - A CT is created and starts at some `start_ts` optionally later dropped and
38//! stopped at some `end_ts`.
39//! - A CT takes one or more _input_s. These must be persist shards (i.e. TABLE,
40//! SOURCE, MV, but not VIEW).
41//! - A CT has one or more _output_s. The outputs are (initially) owned by the
42//! task and cannot be written to by other parts of the system.
43//! - The task is run for each time one of the inputs changes starting at
44//! `start_ts`.
45//! - It is given the changes in its inputs at time `T` as diffs.
46//! - These are presented as two SQL relations with just the inserts/deletes.
47//! - NB: A full collection for the input can always be recovered by also
48//! using the input as a "reference" (see below) and applying the diffs.
49//! - The task logic is expressed as a SQL transaction that does all reads at
50//! commits all writes at `T`
51//! - The notable exception to this is self-referential reads of the CT
52//! output. See below for how that works.
53//! - This logic can _reference_ any nameable object in the system, not just the
54//! inputs.
55//! - However, the logic/transaction can mutate only the outputs.
56//! - Summary of differences between inputs and references:
57//! - The task receives snapshot + changes for references (like regular
58//! dataflow inputs today) but only changes for inputs.
59//! - The task only produces output in response to changes in the inputs but
60//! not in response to changes in the references.
61//! - Instead of re-evaluating the task logic from scratch for each input time,
62//! we maintain the collection representing desired writes to the output(s) as
63//! a dataflow.
64//! - The task dataflow is tied to a `CLUSTER` and runs on each `REPLICA`.
65//! - HA strategy: multi-replica clusters race to commit and the losers throw
66//! away the result.
67//!
68//! ## Self-References
69//!
70//! Self-references must be handled differently from other reads. When computing
71//! the proposed write to some output at `T`, we can only know the contents of
72//! it through `T-1` (the exclusive upper is `T`).
73//!
74//! We address this by initially assuming that the output contains no changes at
75//! `T`, then evaluating each of the statements in order, allowing them to see
76//! the proposed output changes made by the previous statements. By default,
77//! this is stopped after one iteration and proposed output diffs are committed
78//! if possible. (We could also add options for iterating to a fixpoint,
79//! stop/error after N iters, etc.) Then to compute the changes at `T+1`, we
80//! read in what was actually written to the output at `T` (maybe some other
81//! replica wrote something different) and begin again.
82//!
83//! The above is very similar to how timely/differential dataflow iteration
84//! works, except that our feedback loop goes through persist and the loop
85//! timestamp is already `mz_repr::Timestamp`.
86//!
87//! This is implemented as follows:
88//! - `let I = persist_source(self-reference)`
89//! - Transform `I` such that the contents at `T-1` are presented at `T` (i.e.
90//! initially assume `T` is unchanged from `T-1`).
91//! - TODO(ct3): Actually implement the following.
92//! - In an iteration sub-scope:
93//! - Bring `I` into the sub-scope and `let proposed = Variable`.
94//! - We need a collection that at `(T, 0)` is always the contents of `I` at
95//! `T`, but at `(T, 1...)` contains the proposed diffs by the CT logic. We
96//! can construct it by concatenating `I` with `proposed` except that we
97//! also need to retract everything in `proposed` for the next `(T+1, 0)`
98//! (because `I` is the source of truth for what actually committed).
99//! - `let R = retract_at_next_outer_ts(proposed)`
100//! - `let result = logic(concat(I, proposed, R))`
101//! - `proposed.set(result)`
102//! - Then we return `proposed.leave()` for attempted write to persist.
103//!
104//! ## As Ofs and Output Uppers
105//!
106//! - A continual task is first created with an initial as_of `I`. It is
107//! initially rendered at as_of `I==A` but as it makes progress, it may be
108//! rendered at later as_ofs `I<A`.
109//! - It is required that the output collection springs into existence at `I`
110//! (i.e. receives the initial contents at `I`).
111//! - For a snapshot CT, the full contents of the input at `I` are run through
112//! the CT logic and written at `I`.
113//! - For a non-snapshot CT, the collection is defined to be empty at `I`
114//! (i.e. if the input happened to be written exactly at `I`, we'd ignore
115//! it) and then start writing at `I+1`.
116//! - As documented in [DataflowDescription::as_of], `A` is the time we render
117//! the inputs.
118//! - An MV with an as_of of `A` will both have inputs rendered at `A` and
119//! also the first time it could write is also `A`.
120//! - A CT is the same on the initial render (`I==A`), but on renders after it
121//! has made progress (`I<A`) the first time that it could potentially
122//! write is `A+1`. This is because a persist_source started with
123//! SnapshotMode::Exclude can only start emitting diffs at `as_of+1`.
124//! - As a result, we hold back the since on inputs to be strictly less than
125//! the upper of the output. (This is only necessary for CTs, but we also do
126//! it for MVs to avoid the special case.)
127//! - For CT "inputs" (which are disallowed from being the output), we render
128//! the persist_source with as_of `A`.
129//! - When `I==A` we include the snapshot iff the snapshot option is used.
130//! - When `I<A` we always exclude the snapshot. It would be unnecessary and
131//! this is an absolutely critical performance optimization to make CT
132//! rehydration times independent of input size.
133//! - For CT "references", we render the persist_source with as_of `A` and
134//! always include the snapshot.
135//! - There is one subtlety: self-references on the initial render. We need
136//! the contents to be available at `A-1`, so that we can do the
137//! step_forward described above to get it at `A`. However, the collection
138//! springs into existence at `I`, so we when `I==A`, we're not allowed to
139//! read it as_of `A-1` (the since of the shard may have advanced past
140//! that). We address this by rendering the persist_source as normal at
141//! `A`. On startup, persist_source immediately downgrades its frontier to
142//! `A` (making `A-1` readable). Combined with step_forward, this is
143//! enough to unblock the CT self-reference. We do however have to tweak
144//! the `suppress_early_progress` operator to use `A-1` instead of `A` for
145//! this case.
146//! - On subsequent renders, self-references work as normal.
147
148use std::any::Any;
149use std::cell::RefCell;
150use std::collections::BTreeSet;
151use std::rc::Rc;
152use std::sync::Arc;
153
154use differential_dataflow::consolidation::ConsolidatingContainerBuilder;
155use differential_dataflow::difference::Semigroup;
156use differential_dataflow::lattice::Lattice;
157use differential_dataflow::{AsCollection, Hashable, VecCollection};
158use futures::{Future, FutureExt, StreamExt};
159use mz_compute_types::dataflows::DataflowDescription;
160use mz_compute_types::sinks::{ComputeSinkConnection, ComputeSinkDesc, ContinualTaskConnection};
161use mz_ore::cast::CastFrom;
162use mz_ore::collections::HashMap;
163use mz_persist_client::Diagnostics;
164use mz_persist_client::error::UpperMismatch;
165use mz_persist_client::operators::shard_source::SnapshotMode;
166use mz_persist_client::write::WriteHandle;
167use mz_persist_types::codec_impls::UnitSchema;
168use mz_repr::{Diff, GlobalId, Row, Timestamp};
169use mz_storage_types::StorageDiff;
170use mz_storage_types::controller::CollectionMetadata;
171use mz_storage_types::errors::DataflowError;
172use mz_storage_types::sources::SourceData;
173use mz_timely_util::builder_async::{Button, Event, OperatorBuilder as AsyncOperatorBuilder};
174use mz_timely_util::operator::CollectionExt;
175use mz_timely_util::probe;
176use mz_timely_util::probe::ProbeNotify;
177use timely::PartialOrder;
178use timely::dataflow::channels::pact::{Exchange, Pipeline};
179use timely::dataflow::operators::generic::OutputBuilder;
180use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
181use timely::dataflow::operators::vec::{Filter, Map};
182use timely::dataflow::operators::{FrontierNotificator, Operator};
183use timely::dataflow::{ProbeHandle, Scope};
184use timely::progress::frontier::AntichainRef;
185use timely::progress::{Antichain, Timestamp as _};
186use tracing::debug;
187
188use crate::compute_state::ComputeState;
189use crate::render::StartSignal;
190use crate::render::sinks::SinkRender;
191use crate::sink::ConsolidatingVec;
192
193pub(crate) struct ContinualTaskCtx<G: Scope<Timestamp = Timestamp>> {
194 name: Option<String>,
195 dataflow_as_of: Option<Antichain<Timestamp>>,
196 inputs_with_snapshot: Option<bool>,
197 ct_inputs: BTreeSet<GlobalId>,
198 ct_outputs: BTreeSet<GlobalId>,
199 pub ct_times: Vec<VecCollection<G, (), Diff>>,
200}
201
202/// An encapsulation of the transformation logic necessary on data coming into a
203/// continual task.
204///
205/// NB: In continual task jargon, an "input" contains diffs and a "reference" is
206/// a normal source/collection.
207pub(crate) enum ContinualTaskSourceTransformer {
208 /// A collection containing, at each time T, exactly the inserts at time T
209 /// in the transformed collection.
210 ///
211 /// For example:
212 /// - Input: {} at 0, {1} at 1, {1} at 2, ...
213 /// - Output: {} at 0, {1} at 1, {} at 2, ...
214 ///
215 /// We'll presumably have the same for deletes eventually, but it's not
216 /// exposed in the SQL frontend yet.
217 InsertsInput {
218 source_id: GlobalId,
219 with_snapshot: bool,
220 },
221 /// A self-reference to the continual task's output. This is essentially a
222 /// timely feedback loop via the persist shard. See module rustdoc for how
223 /// this works.
224 SelfReference { source_id: GlobalId },
225 /// A normal collection (no-op transformation).
226 NormalReference,
227}
228
229impl ContinualTaskSourceTransformer {
230 /// The persist_source `SnapshotMode` to use when reading this source.
231 pub fn snapshot_mode(&self) -> SnapshotMode {
232 use ContinualTaskSourceTransformer::*;
233 match self {
234 InsertsInput {
235 with_snapshot: false,
236 ..
237 } => SnapshotMode::Exclude,
238 InsertsInput {
239 with_snapshot: true,
240 ..
241 }
242 | SelfReference { .. }
243 | NormalReference => SnapshotMode::Include,
244 }
245 }
246
247 /// Returns the as_of to use with the suppress_early_progress operator for
248 /// this source. See the module rustdoc for context.
249 pub fn suppress_early_progress_as_of(
250 &self,
251 as_of: Antichain<Timestamp>,
252 ) -> Antichain<Timestamp> {
253 use ContinualTaskSourceTransformer::*;
254 match self {
255 InsertsInput { .. } => as_of,
256 SelfReference { .. } => as_of
257 .iter()
258 .map(|x| x.step_back().unwrap_or_else(Timestamp::minimum))
259 .collect(),
260 NormalReference => as_of,
261 }
262 }
263
264 /// Performs the necessary transformation on the source collection.
265 ///
266 /// Returns the transformed "oks" and "errs" collections. Also returns the
267 /// appropriate `ct_times` collection used to inform the sink which times
268 /// were changed in the inputs.
269 pub fn transform<S: Scope<Timestamp = Timestamp>>(
270 &self,
271 oks: VecCollection<S, Row, Diff>,
272 errs: VecCollection<S, DataflowError, Diff>,
273 ) -> (
274 VecCollection<S, Row, Diff>,
275 VecCollection<S, DataflowError, Diff>,
276 VecCollection<S, (), Diff>,
277 ) {
278 use ContinualTaskSourceTransformer::*;
279 match self {
280 // Make a collection s.t, for each time T in the input, the output
281 // contains the inserts at T.
282 InsertsInput { source_id, .. } => {
283 let name = source_id.to_string();
284 // Keep only the inserts.
285 let oks = oks.inner.filter(|(_, _, diff)| diff.is_positive());
286 // Grab the original times for use in the sink operator.
287 let (oks, times) = oks.as_collection().times_extract(&name);
288 // Then retract everything at the next timestamp.
289 let oks = oks.inner.flat_map(|(row, ts, diff)| {
290 let retract_ts = ts.step_forward();
291 let negation = -diff;
292 [(row.clone(), ts, diff), (row, retract_ts, negation)]
293 });
294 (oks.as_collection(), errs, times)
295 }
296 NormalReference => {
297 let times = VecCollection::empty(&oks.scope());
298 (oks, errs, times)
299 }
300 // When computing an self-referential output at `T`, start by
301 // assuming there are no changes from the contents at `T-1`. See the
302 // module rustdoc for how this fits into the larger picture.
303 SelfReference { source_id } => {
304 let name = source_id.to_string();
305 let times = VecCollection::empty(&oks.scope());
306 // step_forward will panic at runtime if it receives a data or
307 // capability with a time that cannot be stepped forward (i.e.
308 // because it is already the max). We're safe here because this
309 // is stepping `T-1` forward to `T`.
310 let oks = oks.step_forward(&name);
311 let errs = errs.step_forward(&name);
312 (oks, errs, times)
313 }
314 }
315 }
316}
317
318impl<G: Scope<Timestamp = Timestamp>> ContinualTaskCtx<G> {
319 pub fn new<P, S>(dataflow: &DataflowDescription<P, S, Timestamp>) -> Self {
320 let mut name = None;
321 let mut ct_inputs = BTreeSet::new();
322 let mut ct_outputs = BTreeSet::new();
323 let mut inputs_with_snapshot = None;
324 for (sink_id, sink) in &dataflow.sink_exports {
325 match &sink.connection {
326 ComputeSinkConnection::ContinualTask(ContinualTaskConnection {
327 input_id, ..
328 }) => {
329 ct_outputs.insert(*sink_id);
330 ct_inputs.insert(*input_id);
331 // There's only one CT sink per dataflow at this point.
332 assert_eq!(name, None);
333 name = Some(sink_id.to_string());
334 assert_eq!(inputs_with_snapshot, None);
335 match (
336 sink.with_snapshot,
337 dataflow.as_of.as_ref(),
338 dataflow.initial_storage_as_of.as_ref(),
339 ) {
340 // User specified no snapshot when creating the CT.
341 (false, _, _) => inputs_with_snapshot = Some(false),
342 // User specified a snapshot but we're past the initial
343 // as_of.
344 (true, Some(as_of), Some(initial_as_of))
345 if PartialOrder::less_than(initial_as_of, as_of) =>
346 {
347 inputs_with_snapshot = Some(false)
348 }
349 // User specified a snapshot and we're either at the
350 // initial creation, or we don't know (builtin CTs). If
351 // we don't know, it's always safe to fall back to
352 // snapshotting, at worst it's wasted work and will get
353 // filtered.
354 (true, _, _) => inputs_with_snapshot = Some(true),
355 }
356 }
357 _ => continue,
358 }
359 }
360 let mut ret = ContinualTaskCtx {
361 name,
362 dataflow_as_of: None,
363 inputs_with_snapshot,
364 ct_inputs,
365 ct_outputs,
366 ct_times: Vec::new(),
367 };
368 // Only clone the as_of if we're in a CT dataflow.
369 if ret.is_ct_dataflow() {
370 ret.dataflow_as_of = dataflow.as_of.clone();
371 // Sanity check that we have a name if we're in a CT dataflow.
372 assert!(ret.name.is_some());
373 }
374 ret
375 }
376
377 pub fn is_ct_dataflow(&self) -> bool {
378 // Inputs are non-empty iff outputs are non-empty.
379 assert_eq!(self.ct_inputs.is_empty(), self.ct_outputs.is_empty());
380 !self.ct_outputs.is_empty()
381 }
382
383 pub fn get_ct_source_transformer(
384 &self,
385 source_id: GlobalId,
386 ) -> Option<ContinualTaskSourceTransformer> {
387 let Some(inputs_with_snapshot) = self.inputs_with_snapshot else {
388 return None;
389 };
390 let transformer = match (
391 self.ct_inputs.contains(&source_id),
392 self.ct_outputs.contains(&source_id),
393 ) {
394 (false, false) => ContinualTaskSourceTransformer::NormalReference,
395 (false, true) => ContinualTaskSourceTransformer::SelfReference { source_id },
396 (true, false) => ContinualTaskSourceTransformer::InsertsInput {
397 source_id,
398 with_snapshot: inputs_with_snapshot,
399 },
400 (true, true) => panic!("ct output is not allowed to be an input"),
401 };
402 Some(transformer)
403 }
404
405 pub fn input_times(&self, scope: &G) -> Option<VecCollection<G, (), Diff>> {
406 // We have a name iff this is a CT dataflow.
407 assert_eq!(self.is_ct_dataflow(), self.name.is_some());
408 let Some(name) = self.name.as_ref() else {
409 return None;
410 };
411 // Note that self.ct_times might be empty (if the user didn't reference
412 // the input), but this still does the correct, though maybe useless,
413 // thing: no diffs coming into the input means no times to write at.
414 let ct_times = differential_dataflow::collection::concatenate(
415 &mut scope.clone(),
416 self.ct_times.iter().cloned(),
417 );
418 // Reduce this down to one update per-time-per-worker before exchanging
419 // it, so we don't waste work on unnecessarily high data volumes.
420 let ct_times = ct_times.times_reduce(name);
421 Some(ct_times)
422 }
423}
424
425impl<G> SinkRender<G> for ContinualTaskConnection<CollectionMetadata>
426where
427 G: Scope<Timestamp = Timestamp>,
428{
429 fn render_sink(
430 &self,
431 compute_state: &mut ComputeState,
432 _sink: &ComputeSinkDesc<CollectionMetadata>,
433 sink_id: GlobalId,
434 as_of: Antichain<Timestamp>,
435 start_signal: StartSignal,
436 oks: VecCollection<G, Row, Diff>,
437 errs: VecCollection<G, DataflowError, Diff>,
438 append_times: Option<VecCollection<G, (), Diff>>,
439 flow_control_probe: &probe::Handle<Timestamp>,
440 ) -> Option<Rc<dyn Any>> {
441 let name = sink_id.to_string();
442
443 let to_append = oks
444 .map(|x| SourceData(Ok(x)))
445 .concat(errs.map(|x| SourceData(Err(x))));
446 let append_times = append_times.expect("should be provided by ContinualTaskCtx");
447
448 let write_handle = {
449 let clients = Arc::clone(&compute_state.persist_clients);
450 let metadata = self.storage_metadata.clone();
451 let handle_purpose = format!("ct_sink({})", name);
452 async move {
453 let client = clients
454 .open(metadata.persist_location)
455 .await
456 .expect("valid location");
457 client
458 .open_writer(
459 metadata.data_shard,
460 metadata.relation_desc.into(),
461 UnitSchema.into(),
462 Diagnostics {
463 shard_name: sink_id.to_string(),
464 handle_purpose,
465 },
466 )
467 .await
468 .expect("codecs should match")
469 }
470 };
471
472 let collection = compute_state.expect_collection_mut(sink_id);
473 let probe = ProbeHandle::default();
474 let to_append = to_append
475 .probe_with(&probe)
476 .inner
477 .probe_notify_with(vec![flow_control_probe.clone()])
478 .as_collection();
479 collection.compute_probe = Some(probe);
480 let sink_write_frontier = Rc::new(RefCell::new(Antichain::from_elem(Timestamp::minimum())));
481 collection.sink_write_frontier = Some(Rc::clone(&sink_write_frontier));
482
483 // TODO(ct1): Obey `compute_state.read_only_rx`
484 //
485 // Seemingly, the read-only env needs to tail the output shard and keep
486 // historical updates around until it sees that the output frontier
487 // advances beyond their times.
488 let sink_button = continual_task_sink(
489 &name,
490 to_append,
491 append_times,
492 as_of,
493 write_handle,
494 start_signal,
495 sink_write_frontier,
496 );
497 Some(Rc::new(sink_button.press_on_drop()))
498 }
499}
500
501fn continual_task_sink<G: Scope<Timestamp = Timestamp>>(
502 name: &str,
503 to_append: VecCollection<G, SourceData, Diff>,
504 append_times: VecCollection<G, (), Diff>,
505 as_of: Antichain<Timestamp>,
506 write_handle: impl Future<Output = WriteHandle<SourceData, (), Timestamp, StorageDiff>>
507 + Send
508 + 'static,
509 start_signal: StartSignal,
510 output_frontier: Rc<RefCell<Antichain<Timestamp>>>,
511) -> Button {
512 let scope = to_append.scope();
513 let mut op = AsyncOperatorBuilder::new(format!("ct_sink({})", name), scope.clone());
514
515 // TODO(ct2): This all works perfectly well data parallel (assuming we
516 // broadcast the append_times). We just need to hook it up to the
517 // multi-worker persist-sink, but that requires some refactoring. This would
518 // also remove the need for this to be an async timely operator.
519 let active_worker = name.hashed();
520 let to_append_input =
521 op.new_input_for_many(to_append.inner, Exchange::new(move |_| active_worker), []);
522 let append_times_input = op.new_input_for_many(
523 append_times.inner,
524 Exchange::new(move |_| active_worker),
525 [],
526 );
527
528 let active_worker = usize::cast_from(active_worker) % scope.peers() == scope.index();
529 let button = op.build(move |_capabilities| async move {
530 if !active_worker {
531 output_frontier.borrow_mut().clear();
532 return;
533 }
534
535 // SUBTLE: The start_signal below may not be unblocked by the compute
536 // controller until it thinks the inputs are "ready" (i.e. readable at
537 // the as_of), but if the CT is self-referential, one of the inputs will
538 // be the output (which starts at `T::minimum()`, not the as_of). To
539 // break this cycle, before we even get the start signal, go ahead and
540 // advance the output's (exclusive) upper to the first time that this CT
541 // might write: `as_of+1`. Because we don't want this to happen on
542 // restarts, only do it if the upper is `T::minimum()`.
543 let mut write_handle = write_handle.await;
544 {
545 let res = write_handle
546 .compare_and_append_batch(
547 &mut [],
548 Antichain::from_elem(Timestamp::minimum()),
549 as_of.clone(),
550 true,
551 )
552 .await
553 .expect("usage was valid");
554 match res {
555 // We advanced the upper.
556 Ok(()) => {}
557 // Someone else advanced the upper.
558 Err(UpperMismatch { .. }) => {}
559 }
560 }
561
562 let () = start_signal.await;
563
564 #[derive(Debug)]
565 enum OpEvent<C> {
566 ToAppend(Event<Timestamp, C, Vec<(SourceData, Timestamp, Diff)>>),
567 AppendTimes(Event<Timestamp, C, Vec<((), Timestamp, Diff)>>),
568 }
569
570 impl<C: std::fmt::Debug> OpEvent<C> {
571 fn apply(self, state: &mut SinkState<SourceData, Timestamp>) {
572 debug!("ct_sink event {:?}", self);
573 match self {
574 OpEvent::ToAppend(Event::Data(_cap, x)) => {
575 for (k, t, d) in x {
576 state.to_append.push(((k, t), d));
577 }
578 }
579 OpEvent::ToAppend(Event::Progress(x)) => state.to_append_progress = x,
580 OpEvent::AppendTimes(Event::Data(_cap, x)) => state
581 .append_times
582 .extend(x.into_iter().map(|((), t, _d)| t)),
583 OpEvent::AppendTimes(Event::Progress(x)) => state.append_times_progress = x,
584 }
585 }
586 }
587
588 let to_insert_input = to_append_input.map(OpEvent::ToAppend);
589 let append_times_input = append_times_input.map(OpEvent::AppendTimes);
590 let mut op_inputs = futures::stream::select(to_insert_input, append_times_input);
591
592 let mut state = SinkState::new();
593 loop {
594 // Loop until we've processed all the work we can.
595 loop {
596 if PartialOrder::less_than(&*output_frontier.borrow(), &state.output_progress) {
597 output_frontier.borrow_mut().clear();
598 output_frontier
599 .borrow_mut()
600 .extend(state.output_progress.iter().cloned());
601 }
602
603 debug!("ct_sink about to process {:?}", state);
604 let Some((new_upper, to_append)) = state.process() else {
605 break;
606 };
607 debug!("ct_sink got write {:?}: {:?}", new_upper, to_append);
608 state.output_progress =
609 truncating_compare_and_append(&mut write_handle, to_append, new_upper).await;
610 }
611
612 // Then try to generate some more work by reading inputs.
613 let Some(event) = op_inputs.next().await else {
614 // Inputs exhausted, shutting down.
615 output_frontier.borrow_mut().clear();
616 return;
617 };
618 event.apply(&mut state);
619 // Also drain any other events that may be ready.
620 while let Some(Some(event)) = op_inputs.next().now_or_never() {
621 event.apply(&mut state);
622 }
623 }
624 });
625
626 button
627}
628
629/// Writes the given data to the shard, truncating it as necessary.
630///
631/// Returns the latest known upper for the shard.
632async fn truncating_compare_and_append(
633 write_handle: &mut WriteHandle<SourceData, (), Timestamp, StorageDiff>,
634 to_append: Vec<((&SourceData, &()), &Timestamp, StorageDiff)>,
635 new_upper: Antichain<Timestamp>,
636) -> Antichain<Timestamp> {
637 let mut expected_upper = write_handle.shared_upper();
638 loop {
639 if !PartialOrder::less_than(&expected_upper, &new_upper) {
640 debug!("ct_sink skipping {:?}", new_upper.elements());
641 return expected_upper;
642 }
643 let res = write_handle
644 .compare_and_append(&to_append, expected_upper.clone(), new_upper.clone())
645 .await
646 .expect("usage was valid");
647 debug!(
648 "ct_sink write res {:?}-{:?}: {:?}",
649 expected_upper.elements(),
650 new_upper.elements(),
651 res
652 );
653 match res {
654 Ok(()) => return new_upper,
655 Err(err) => {
656 expected_upper = err.current;
657 continue;
658 }
659 }
660 }
661}
662
663#[derive(Debug)]
664struct SinkState<D, T> {
665 /// The known times at which we're going to write data to the output. This
666 /// is guaranteed to include all times < append_times_progress, except that
667 /// ones < output_progress may have been truncated.
668 append_times: BTreeSet<T>,
669 append_times_progress: Antichain<T>,
670
671 /// The data we've collected to append to the output. This is often
672 /// compacted to advancing times and is expected to be ~empty in the steady
673 /// state.
674 to_append: ConsolidatingVec<(D, T)>,
675 to_append_progress: Antichain<T>,
676
677 /// A lower bound on the upper of the output.
678 output_progress: Antichain<T>,
679}
680
681impl<D: Ord> SinkState<D, Timestamp> {
682 fn new() -> Self {
683 SinkState {
684 append_times: BTreeSet::new(),
685 append_times_progress: Antichain::from_elem(Timestamp::minimum()),
686 to_append: ConsolidatingVec::new(128, 0),
687 to_append_progress: Antichain::from_elem(Timestamp::minimum()),
688 output_progress: Antichain::from_elem(Timestamp::minimum()),
689 }
690 }
691
692 /// Returns data to write to the output, if any, and the new upper to use.
693 fn process(
694 &mut self,
695 ) -> Option<(
696 Antichain<Timestamp>,
697 Vec<((&D, &()), &Timestamp, StorageDiff)>,
698 )> {
699 // We can only append at times >= the output_progress, so pop off
700 // anything unnecessary.
701 while let Some(x) = self.append_times.first() {
702 if self.output_progress.less_equal(x) {
703 break;
704 }
705 self.append_times.pop_first();
706 }
707
708 // Find the smallest append_time before append_time_progress. This is
709 // the next time we might need to write data at. Note that we can only
710 // act on append_times once the progress has passed them, because they
711 // could come out of order.
712 let write_ts = match self.append_times.first() {
713 Some(x) if !self.append_times_progress.less_equal(x) => x,
714 Some(_) | None => {
715 // The CT sink's contract is that it only writes data at times
716 // we received an input diff. There are none in
717 // `[output_progress, append_times_progress)`, so we can go
718 // ahead and advance the upper of the output, if it's not
719 // already.
720 //
721 // We could instead ensure liveness by basing this off of
722 // to_append, but for any CTs reading the output (expected to be
723 // a common case) we'd end up looping each timestamp through
724 // persist one-by-one.
725 if PartialOrder::less_than(&self.output_progress, &self.append_times_progress) {
726 return Some((self.append_times_progress.clone(), Vec::new()));
727 }
728 // Otherwise, nothing to do!
729 return None;
730 }
731 };
732
733 if self.to_append_progress.less_equal(write_ts) {
734 // Don't have all the necessary data yet.
735 if self.output_progress.less_than(write_ts) {
736 // We can advance the output upper up to the write_ts. For
737 // self-referential CTs this might be necessary to ensure
738 // dataflow progress.
739 return Some((Antichain::from_elem(write_ts.clone()), Vec::new()));
740 }
741 return None;
742 }
743
744 // Time to write some data! Produce the collection as of write_ts by
745 // advancing timestamps, consolidating, and filtering out anything at
746 // future timestamps.
747 let as_of = std::slice::from_ref(write_ts);
748 for ((_, t), _) in self.to_append.iter_mut() {
749 t.advance_by(AntichainRef::new(as_of))
750 }
751 // TODO(ct2): Metrics for vec len and cap.
752 self.to_append.consolidate();
753
754 let append_data = self
755 .to_append
756 .iter()
757 .filter_map(|((k, t), d)| (t <= write_ts).then_some(((k, &()), t, d.into_inner())))
758 .collect();
759 Some((Antichain::from_elem(write_ts.step_forward()), append_data))
760 }
761}
762
763trait StepForward<G: Scope, D, R> {
764 /// Translates a collection one timestamp "forward" (i.e. `T` -> `T+1` as
765 /// defined by `TimestampManipulation::step_forward`).
766 ///
767 /// This includes:
768 /// - The differential timestamps in each data.
769 /// - The capabilities paired with that data.
770 /// - (As a consequence of the previous) the output frontier is one step forward
771 /// of the input frontier.
772 ///
773 /// The caller is responsible for ensuring that all data and capabilities given
774 /// to this operator can be stepped forward without panicking, otherwise the
775 /// operator will panic at runtime.
776 fn step_forward(self, name: &str) -> VecCollection<G, D, R>;
777}
778
779impl<G, D, R> StepForward<G, D, R> for VecCollection<G, D, R>
780where
781 G: Scope<Timestamp = Timestamp>,
782 D: Clone + 'static,
783 R: Semigroup + 'static,
784{
785 fn step_forward(self, name: &str) -> VecCollection<G, D, R> {
786 let name = format!("ct_step_forward({})", name);
787 let mut builder = OperatorBuilder::new(name, self.scope());
788 let (output, output_stream) = builder.new_output();
789 let mut output = OutputBuilder::from(output);
790
791 // We step forward (by one) each data timestamp and capability. As a
792 // result the output's frontier is guaranteed to be one past the input
793 // frontier, so make this promise to timely.
794 let step_forward_summary = Timestamp::from(1);
795 let mut input = builder.new_input_connection(
796 self.inner,
797 Pipeline,
798 [(0, Antichain::from_elem(step_forward_summary))],
799 );
800 builder.set_notify(false);
801 builder.build(move |_caps| {
802 move |_frontiers| {
803 let mut output = output.activate();
804 input.for_each(|cap, data| {
805 for (_, ts, _) in data.iter_mut() {
806 *ts = ts.step_forward();
807 }
808 let cap = cap.delayed(&cap.time().step_forward(), 0);
809 output.session(&cap).give_container(data);
810 });
811 }
812 });
813
814 output_stream.as_collection()
815 }
816}
817
818trait TimesExtract<G: Scope, D, R> {
819 /// Returns a collection with the times changed in the input collection.
820 ///
821 /// This works by mapping the data piece of the differential tuple to `()`.
822 /// It is essentially the same as the following, but without cloning
823 /// everything in the input.
824 ///
825 /// ```ignore
826 /// input.map(|(_data, ts, diff)| ((), ts, diff))
827 /// ```
828 ///
829 /// The output may be partially consolidated, but no consolidation
830 /// guarantees are made.
831 fn times_extract(self, name: &str) -> (VecCollection<G, D, R>, VecCollection<G, (), R>);
832}
833
834impl<G, D, R> TimesExtract<G, D, R> for VecCollection<G, D, R>
835where
836 G: Scope<Timestamp = Timestamp>,
837 D: Clone + 'static,
838 R: Semigroup + 'static + std::fmt::Debug,
839{
840 fn times_extract(self, name: &str) -> (VecCollection<G, D, R>, VecCollection<G, (), R>) {
841 let name = format!("ct_times_extract({})", name);
842 let mut builder = OperatorBuilder::new(name, self.scope());
843 let (passthrough, passthrough_stream) = builder.new_output();
844 let mut passthrough = OutputBuilder::from(passthrough);
845 let (times, times_stream) = builder.new_output();
846 let mut times = OutputBuilder::<_, ConsolidatingContainerBuilder<_>>::from(times);
847 let mut input = builder.new_input(self.inner, Pipeline);
848 builder.set_notify(false);
849 builder.build(|_caps| {
850 move |_frontiers| {
851 let mut passthrough = passthrough.activate();
852 let mut times = times.activate();
853 input.for_each_time(|time, data| {
854 let mut times_session = times.session_with_builder(&time);
855 let mut passthrough_session = passthrough.session(&time);
856 for data in data {
857 let times_iter =
858 data.iter().map(|(_data, ts, diff)| ((), *ts, diff.clone()));
859 times_session.give_iterator(times_iter);
860 passthrough_session.give_container(data);
861 }
862 });
863 }
864 });
865 (
866 passthrough_stream.as_collection(),
867 times_stream.as_collection(),
868 )
869 }
870}
871
872trait TimesReduce<G: Scope, R> {
873 /// This is essentially a specialized impl of consolidate, with a HashMap
874 /// instead of the Trace.
875 fn times_reduce(self, name: &str) -> VecCollection<G, (), R>;
876}
877
878impl<G, R> TimesReduce<G, R> for VecCollection<G, (), R>
879where
880 G: Scope<Timestamp = Timestamp>,
881 R: Semigroup + 'static + std::fmt::Debug,
882{
883 fn times_reduce(self, name: &str) -> VecCollection<G, (), R> {
884 let name = format!("ct_times_reduce({})", name);
885 self.inner
886 .unary_frontier(Pipeline, &name, |_caps, _info| {
887 let mut notificator = FrontierNotificator::default();
888 let mut stash = HashMap::<_, R>::new();
889 move |(input, frontier), output| {
890 input.for_each(|cap, data| {
891 for ((), ts, diff) in data.drain(..) {
892 notificator.notify_at(cap.delayed(&ts, 0));
893 if let Some(sum) = stash.get_mut(&ts) {
894 sum.plus_equals(&diff);
895 } else {
896 stash.insert(ts, diff);
897 }
898 }
899 });
900 notificator.for_each(&[frontier], |cap, _not| {
901 if let Some(diff) = stash.remove(cap.time()) {
902 output.session(&cap).give(((), cap.time().clone(), diff));
903 }
904 });
905 }
906 })
907 .as_collection()
908 }
909}
910
911#[cfg(test)]
912mod tests {
913 use differential_dataflow::AsCollection;
914 use mz_repr::Timestamp;
915 use timely::Config;
916 use timely::dataflow::ProbeHandle;
917 use timely::dataflow::operators::capture::Extract;
918 use timely::dataflow::operators::{Capture, Input, ToStream};
919 use timely::progress::Antichain;
920
921 use super::*;
922
923 #[mz_ore::test]
924 fn step_forward() {
925 timely::execute(Config::thread(), |worker| {
926 let (mut input, probe, output) = worker.dataflow(|scope| {
927 let (handle, input) = scope.new_input();
928 let probe = ProbeHandle::<Timestamp>::new();
929 let output = input
930 .as_collection()
931 .step_forward("test")
932 .probe_with(&probe)
933 .inner
934 .capture();
935 (handle, probe, output)
936 });
937
938 let mut expected = Vec::new();
939 for i in 0u64..10 {
940 let in_ts = Timestamp::new(i);
941 let out_ts = in_ts.step_forward();
942 input.send((i, in_ts, 1));
943 input.advance_to(in_ts.step_forward());
944
945 // We should get the data out advanced by `step_forward` and
946 // also, crucially, the output frontier should do the same (i.e.
947 // this is why we can't simply use `VecCollection::delay`).
948 worker.step_while(|| probe.less_than(&out_ts.step_forward()));
949 expected.push((i, out_ts, 1));
950 }
951 // Closing the input should allow the output to advance and the
952 // dataflow to shut down.
953 input.close();
954 while worker.step() {}
955
956 let actual = output
957 .extract()
958 .into_iter()
959 .flat_map(|x| x.1)
960 .collect::<Vec<_>>();
961 assert_eq!(actual, expected);
962 })
963 .unwrap();
964 }
965
966 #[mz_ore::test]
967 fn times_extract() {
968 struct PanicOnClone;
969
970 impl Clone for PanicOnClone {
971 fn clone(&self) -> Self {
972 panic!("boom")
973 }
974 }
975
976 let output = timely::execute_directly(|worker| {
977 worker.dataflow(|scope| {
978 let input = [
979 (PanicOnClone, Timestamp::new(0), 0),
980 (PanicOnClone, Timestamp::new(1), 1),
981 (PanicOnClone, Timestamp::new(1), 1),
982 (PanicOnClone, Timestamp::new(2), -2),
983 (PanicOnClone, Timestamp::new(2), 1),
984 ]
985 .to_stream(scope)
986 .as_collection();
987 let (_passthrough, times) = input.times_extract("test");
988 times.inner.capture()
989 })
990 });
991 let expected = vec![((), Timestamp::new(1), 2), ((), Timestamp::new(2), -1)];
992 let actual = output
993 .extract()
994 .into_iter()
995 .flat_map(|x| x.1)
996 .collect::<Vec<_>>();
997 assert_eq!(actual, expected);
998 }
999
1000 #[mz_ore::test]
1001 fn times_reduce() {
1002 let output = timely::execute_directly(|worker| {
1003 worker.dataflow(|scope| {
1004 let input = [
1005 ((), Timestamp::new(3), 1),
1006 ((), Timestamp::new(2), 1),
1007 ((), Timestamp::new(1), 1),
1008 ((), Timestamp::new(2), 1),
1009 ((), Timestamp::new(3), 1),
1010 ((), Timestamp::new(3), 1),
1011 ]
1012 .to_stream(scope)
1013 .as_collection();
1014 input.times_reduce("test").inner.capture()
1015 })
1016 });
1017 let expected = vec![
1018 ((), Timestamp::new(1), 1),
1019 ((), Timestamp::new(2), 2),
1020 ((), Timestamp::new(3), 3),
1021 ];
1022 let actual = output
1023 .extract()
1024 .into_iter()
1025 .flat_map(|x| x.1)
1026 .collect::<Vec<_>>();
1027 assert_eq!(actual, expected);
1028 }
1029
1030 #[mz_ore::test]
1031 fn ct_sink_state() {
1032 #[track_caller]
1033 fn assert_noop(state: &mut super::SinkState<&'static str, Timestamp>) {
1034 if let Some(ret) = state.process() {
1035 panic!("should be nothing to write: {:?}", ret);
1036 }
1037 }
1038
1039 #[track_caller]
1040 fn assert_write(
1041 state: &mut super::SinkState<&'static str, Timestamp>,
1042 expected_upper: u64,
1043 expected_append: &[&str],
1044 ) {
1045 let (new_upper, to_append) = state.process().expect("should be something to write");
1046 assert_eq!(
1047 new_upper,
1048 Antichain::from_elem(Timestamp::new(expected_upper))
1049 );
1050 let to_append = to_append
1051 .into_iter()
1052 .map(|((k, ()), _ts, _diff)| *k)
1053 .collect::<Vec<_>>();
1054 assert_eq!(to_append, expected_append);
1055 }
1056
1057 let mut s = super::SinkState::new();
1058
1059 // Nothing to do at the initial state.
1060 assert_noop(&mut s);
1061
1062 // Getting data to append is not enough to do anything yet.
1063 s.to_append.push((("a", 1.into()), Diff::ONE));
1064 s.to_append.push((("b", 1.into()), Diff::ONE));
1065 assert_noop(&mut s);
1066
1067 // Knowing that this is the only data we'll get for that timestamp is
1068 // still not enough.
1069 s.to_append_progress = Antichain::from_elem(2.into());
1070 assert_noop(&mut s);
1071
1072 // Even knowing that we got input at that time is not quite enough yet
1073 // (we could be getting these out of order).
1074 s.append_times.insert(1.into());
1075 assert_noop(&mut s);
1076
1077 // Indeed, it did come out of order. Also note that this checks the ==
1078 // case for time vs progress.
1079 s.append_times.insert(0.into());
1080 assert_noop(&mut s);
1081
1082 // Okay, now we know that we've seen all the times we got input up to 3.
1083 // This is enough to allow the empty write of `[0,1)`.
1084 s.append_times_progress = Antichain::from_elem(3.into());
1085 assert_write(&mut s, 1, &[]);
1086
1087 // That succeeded, now we can write the data at 1.
1088 s.output_progress = Antichain::from_elem(1.into());
1089 assert_write(&mut s, 2, &["a", "b"]);
1090
1091 // That succeeded, now we know about some empty time.
1092 s.output_progress = Antichain::from_elem(2.into());
1093 assert_write(&mut s, 3, &[]);
1094
1095 // That succeeded, now nothing to do.
1096 s.output_progress = Antichain::from_elem(3.into());
1097 assert_noop(&mut s);
1098
1099 // Find out about a new time to write at. Even without the data, we can
1100 // do an empty write up to that time.
1101 s.append_times.insert(5.into());
1102 s.append_times_progress = Antichain::from_elem(6.into());
1103 assert_write(&mut s, 5, &[]);
1104
1105 // That succeeded, now nothing to do again.
1106 s.output_progress = Antichain::from_elem(5.into());
1107
1108 // Retract one of the things currently in the collection and add a new
1109 // thing, to verify the consolidate.
1110 s.to_append.push((("a", 5.into()), Diff::MINUS_ONE));
1111 s.to_append.push((("c", 5.into()), Diff::ONE));
1112 s.to_append_progress = Antichain::from_elem(6.into());
1113 assert_write(&mut s, 6, &["b", "c"]);
1114 }
1115}