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::dataflow::channels::pact::{Exchange, Pipeline};
178use timely::dataflow::operators::generic::OutputBuilder;
179use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
180use timely::dataflow::operators::{Filter, FrontierNotificator, Map, Operator};
181use timely::dataflow::{ProbeHandle, Scope};
182use timely::progress::frontier::AntichainRef;
183use timely::progress::{Antichain, Timestamp as _};
184use timely::{Data, PartialOrder};
185use tracing::debug;
186
187use crate::compute_state::ComputeState;
188use crate::render::StartSignal;
189use crate::render::sinks::SinkRender;
190use crate::sink::ConsolidatingVec;
191
192pub(crate) struct ContinualTaskCtx<G: Scope<Timestamp = Timestamp>> {
193    name: Option<String>,
194    dataflow_as_of: Option<Antichain<Timestamp>>,
195    inputs_with_snapshot: Option<bool>,
196    ct_inputs: BTreeSet<GlobalId>,
197    ct_outputs: BTreeSet<GlobalId>,
198    pub ct_times: Vec<VecCollection<G, (), Diff>>,
199}
200
201/// An encapsulation of the transformation logic necessary on data coming into a
202/// continual task.
203///
204/// NB: In continual task jargon, an "input" contains diffs and a "reference" is
205/// a normal source/collection.
206pub(crate) enum ContinualTaskSourceTransformer {
207    /// A collection containing, at each time T, exactly the inserts at time T
208    /// in the transformed collection.
209    ///
210    /// For example:
211    /// - Input: {} at 0, {1} at 1, {1} at 2, ...
212    /// - Output: {} at 0, {1} at 1, {} at 2, ...
213    ///
214    /// We'll presumably have the same for deletes eventually, but it's not
215    /// exposed in the SQL frontend yet.
216    InsertsInput {
217        source_id: GlobalId,
218        with_snapshot: bool,
219    },
220    /// A self-reference to the continual task's output. This is essentially a
221    /// timely feedback loop via the persist shard. See module rustdoc for how
222    /// this works.
223    SelfReference { source_id: GlobalId },
224    /// A normal collection (no-op transformation).
225    NormalReference,
226}
227
228impl ContinualTaskSourceTransformer {
229    /// The persist_source `SnapshotMode` to use when reading this source.
230    pub fn snapshot_mode(&self) -> SnapshotMode {
231        use ContinualTaskSourceTransformer::*;
232        match self {
233            InsertsInput {
234                with_snapshot: false,
235                ..
236            } => SnapshotMode::Exclude,
237            InsertsInput {
238                with_snapshot: true,
239                ..
240            }
241            | SelfReference { .. }
242            | NormalReference => SnapshotMode::Include,
243        }
244    }
245
246    /// Returns the as_of to use with the suppress_early_progress operator for
247    /// this source. See the module rustdoc for context.
248    pub fn suppress_early_progress_as_of(
249        &self,
250        as_of: Antichain<Timestamp>,
251    ) -> Antichain<Timestamp> {
252        use ContinualTaskSourceTransformer::*;
253        match self {
254            InsertsInput { .. } => as_of,
255            SelfReference { .. } => as_of
256                .iter()
257                .map(|x| x.step_back().unwrap_or_else(Timestamp::minimum))
258                .collect(),
259            NormalReference => as_of,
260        }
261    }
262
263    /// Performs the necessary transformation on the source collection.
264    ///
265    /// Returns the transformed "oks" and "errs" collections. Also returns the
266    /// appropriate `ct_times` collection used to inform the sink which times
267    /// were changed in the inputs.
268    pub fn transform<S: Scope<Timestamp = Timestamp>>(
269        &self,
270        oks: VecCollection<S, Row, Diff>,
271        errs: VecCollection<S, DataflowError, Diff>,
272    ) -> (
273        VecCollection<S, Row, Diff>,
274        VecCollection<S, DataflowError, Diff>,
275        VecCollection<S, (), Diff>,
276    ) {
277        use ContinualTaskSourceTransformer::*;
278        match self {
279            // Make a collection s.t, for each time T in the input, the output
280            // contains the inserts at T.
281            InsertsInput { source_id, .. } => {
282                let name = source_id.to_string();
283                // Keep only the inserts.
284                let oks = oks.inner.filter(|(_, _, diff)| diff.is_positive());
285                // Grab the original times for use in the sink operator.
286                let (oks, times) = oks.as_collection().times_extract(&name);
287                // Then retract everything at the next timestamp.
288                let oks = oks.inner.flat_map(|(row, ts, diff)| {
289                    let retract_ts = ts.step_forward();
290                    let negation = -diff;
291                    [(row.clone(), ts, diff), (row, retract_ts, negation)]
292                });
293                (oks.as_collection(), errs, times)
294            }
295            NormalReference => {
296                let times = VecCollection::empty(&oks.scope());
297                (oks, errs, times)
298            }
299            // When computing an self-referential output at `T`, start by
300            // assuming there are no changes from the contents at `T-1`. See the
301            // module rustdoc for how this fits into the larger picture.
302            SelfReference { source_id } => {
303                let name = source_id.to_string();
304                let times = VecCollection::empty(&oks.scope());
305                // step_forward will panic at runtime if it receives a data or
306                // capability with a time that cannot be stepped forward (i.e.
307                // because it is already the max). We're safe here because this
308                // is stepping `T-1` forward to `T`.
309                let oks = oks.step_forward(&name);
310                let errs = errs.step_forward(&name);
311                (oks, errs, times)
312            }
313        }
314    }
315}
316
317impl<G: Scope<Timestamp = Timestamp>> ContinualTaskCtx<G> {
318    pub fn new<P, S>(dataflow: &DataflowDescription<P, S, Timestamp>) -> Self {
319        let mut name = None;
320        let mut ct_inputs = BTreeSet::new();
321        let mut ct_outputs = BTreeSet::new();
322        let mut inputs_with_snapshot = None;
323        for (sink_id, sink) in &dataflow.sink_exports {
324            match &sink.connection {
325                ComputeSinkConnection::ContinualTask(ContinualTaskConnection {
326                    input_id, ..
327                }) => {
328                    ct_outputs.insert(*sink_id);
329                    ct_inputs.insert(*input_id);
330                    // There's only one CT sink per dataflow at this point.
331                    assert_eq!(name, None);
332                    name = Some(sink_id.to_string());
333                    assert_eq!(inputs_with_snapshot, None);
334                    match (
335                        sink.with_snapshot,
336                        dataflow.as_of.as_ref(),
337                        dataflow.initial_storage_as_of.as_ref(),
338                    ) {
339                        // User specified no snapshot when creating the CT.
340                        (false, _, _) => inputs_with_snapshot = Some(false),
341                        // User specified a snapshot but we're past the initial
342                        // as_of.
343                        (true, Some(as_of), Some(initial_as_of))
344                            if PartialOrder::less_than(initial_as_of, as_of) =>
345                        {
346                            inputs_with_snapshot = Some(false)
347                        }
348                        // User specified a snapshot and we're either at the
349                        // initial creation, or we don't know (builtin CTs). If
350                        // we don't know, it's always safe to fall back to
351                        // snapshotting, at worst it's wasted work and will get
352                        // filtered.
353                        (true, _, _) => inputs_with_snapshot = Some(true),
354                    }
355                }
356                _ => continue,
357            }
358        }
359        let mut ret = ContinualTaskCtx {
360            name,
361            dataflow_as_of: None,
362            inputs_with_snapshot,
363            ct_inputs,
364            ct_outputs,
365            ct_times: Vec::new(),
366        };
367        // Only clone the as_of if we're in a CT dataflow.
368        if ret.is_ct_dataflow() {
369            ret.dataflow_as_of = dataflow.as_of.clone();
370            // Sanity check that we have a name if we're in a CT dataflow.
371            assert!(ret.name.is_some());
372        }
373        ret
374    }
375
376    pub fn is_ct_dataflow(&self) -> bool {
377        // Inputs are non-empty iff outputs are non-empty.
378        assert_eq!(self.ct_inputs.is_empty(), self.ct_outputs.is_empty());
379        !self.ct_outputs.is_empty()
380    }
381
382    pub fn get_ct_source_transformer(
383        &self,
384        source_id: GlobalId,
385    ) -> Option<ContinualTaskSourceTransformer> {
386        let Some(inputs_with_snapshot) = self.inputs_with_snapshot else {
387            return None;
388        };
389        let transformer = match (
390            self.ct_inputs.contains(&source_id),
391            self.ct_outputs.contains(&source_id),
392        ) {
393            (false, false) => ContinualTaskSourceTransformer::NormalReference,
394            (false, true) => ContinualTaskSourceTransformer::SelfReference { source_id },
395            (true, false) => ContinualTaskSourceTransformer::InsertsInput {
396                source_id,
397                with_snapshot: inputs_with_snapshot,
398            },
399            (true, true) => panic!("ct output is not allowed to be an input"),
400        };
401        Some(transformer)
402    }
403
404    pub fn input_times(&self, scope: &G) -> Option<VecCollection<G, (), Diff>> {
405        // We have a name iff this is a CT dataflow.
406        assert_eq!(self.is_ct_dataflow(), self.name.is_some());
407        let Some(name) = self.name.as_ref() else {
408            return None;
409        };
410        // Note that self.ct_times might be empty (if the user didn't reference
411        // the input), but this still does the correct, though maybe useless,
412        // thing: no diffs coming into the input means no times to write at.
413        let ct_times = differential_dataflow::collection::concatenate(
414            &mut scope.clone(),
415            self.ct_times.iter().cloned(),
416        );
417        // Reduce this down to one update per-time-per-worker before exchanging
418        // it, so we don't waste work on unnecessarily high data volumes.
419        let ct_times = ct_times.times_reduce(name);
420        Some(ct_times)
421    }
422}
423
424impl<G> SinkRender<G> for ContinualTaskConnection<CollectionMetadata>
425where
426    G: Scope<Timestamp = Timestamp>,
427{
428    fn render_sink(
429        &self,
430        compute_state: &mut ComputeState,
431        _sink: &ComputeSinkDesc<CollectionMetadata>,
432        sink_id: GlobalId,
433        as_of: Antichain<Timestamp>,
434        start_signal: StartSignal,
435        oks: VecCollection<G, Row, Diff>,
436        errs: VecCollection<G, DataflowError, Diff>,
437        append_times: Option<VecCollection<G, (), Diff>>,
438        flow_control_probe: &probe::Handle<Timestamp>,
439    ) -> Option<Rc<dyn Any>> {
440        let name = sink_id.to_string();
441
442        let to_append = oks
443            .map(|x| SourceData(Ok(x)))
444            .concat(&errs.map(|x| SourceData(Err(x))));
445        let append_times = append_times.expect("should be provided by ContinualTaskCtx");
446
447        let write_handle = {
448            let clients = Arc::clone(&compute_state.persist_clients);
449            let metadata = self.storage_metadata.clone();
450            let handle_purpose = format!("ct_sink({})", name);
451            async move {
452                let client = clients
453                    .open(metadata.persist_location)
454                    .await
455                    .expect("valid location");
456                client
457                    .open_writer(
458                        metadata.data_shard,
459                        metadata.relation_desc.into(),
460                        UnitSchema.into(),
461                        Diagnostics {
462                            shard_name: sink_id.to_string(),
463                            handle_purpose,
464                        },
465                    )
466                    .await
467                    .expect("codecs should match")
468            }
469        };
470
471        let collection = compute_state.expect_collection_mut(sink_id);
472        let probe = ProbeHandle::default();
473        let to_append = to_append
474            .probe_with(&probe)
475            .inner
476            .probe_notify_with(vec![flow_control_probe.clone()])
477            .as_collection();
478        collection.compute_probe = Some(probe);
479        let sink_write_frontier = Rc::new(RefCell::new(Antichain::from_elem(Timestamp::minimum())));
480        collection.sink_write_frontier = Some(Rc::clone(&sink_write_frontier));
481
482        // TODO(ct1): Obey `compute_state.read_only_rx`
483        //
484        // Seemingly, the read-only env needs to tail the output shard and keep
485        // historical updates around until it sees that the output frontier
486        // advances beyond their times.
487        let sink_button = continual_task_sink(
488            &name,
489            to_append,
490            append_times,
491            as_of,
492            write_handle,
493            start_signal,
494            sink_write_frontier,
495        );
496        Some(Rc::new(sink_button.press_on_drop()))
497    }
498}
499
500fn continual_task_sink<G: Scope<Timestamp = Timestamp>>(
501    name: &str,
502    to_append: VecCollection<G, SourceData, Diff>,
503    append_times: VecCollection<G, (), Diff>,
504    as_of: Antichain<Timestamp>,
505    write_handle: impl Future<Output = WriteHandle<SourceData, (), Timestamp, StorageDiff>>
506    + Send
507    + 'static,
508    start_signal: StartSignal,
509    output_frontier: Rc<RefCell<Antichain<Timestamp>>>,
510) -> Button {
511    let scope = to_append.scope();
512    let mut op = AsyncOperatorBuilder::new(format!("ct_sink({})", name), scope.clone());
513
514    // TODO(ct2): This all works perfectly well data parallel (assuming we
515    // broadcast the append_times). We just need to hook it up to the
516    // multi-worker persist-sink, but that requires some refactoring. This would
517    // also remove the need for this to be an async timely operator.
518    let active_worker = name.hashed();
519    let to_append_input =
520        op.new_input_for_many(&to_append.inner, Exchange::new(move |_| active_worker), []);
521    let append_times_input = op.new_input_for_many(
522        &append_times.inner,
523        Exchange::new(move |_| active_worker),
524        [],
525    );
526
527    let active_worker = usize::cast_from(active_worker) % scope.peers() == scope.index();
528    let button = op.build(move |_capabilities| async move {
529        if !active_worker {
530            output_frontier.borrow_mut().clear();
531            return;
532        }
533
534        // SUBTLE: The start_signal below may not be unblocked by the compute
535        // controller until it thinks the inputs are "ready" (i.e. readable at
536        // the as_of), but if the CT is self-referential, one of the inputs will
537        // be the output (which starts at `T::minimum()`, not the as_of). To
538        // break this cycle, before we even get the start signal, go ahead and
539        // advance the output's (exclusive) upper to the first time that this CT
540        // might write: `as_of+1`. Because we don't want this to happen on
541        // restarts, only do it if the upper is `T::minimum()`.
542        let mut write_handle = write_handle.await;
543        {
544            let res = write_handle
545                .compare_and_append_batch(
546                    &mut [],
547                    Antichain::from_elem(Timestamp::minimum()),
548                    as_of.clone(),
549                    true,
550                )
551                .await
552                .expect("usage was valid");
553            match res {
554                // We advanced the upper.
555                Ok(()) => {}
556                // Someone else advanced the upper.
557                Err(UpperMismatch { .. }) => {}
558            }
559        }
560
561        let () = start_signal.await;
562
563        #[derive(Debug)]
564        enum OpEvent<C> {
565            ToAppend(Event<Timestamp, C, Vec<(SourceData, Timestamp, Diff)>>),
566            AppendTimes(Event<Timestamp, C, Vec<((), Timestamp, Diff)>>),
567        }
568
569        impl<C: std::fmt::Debug> OpEvent<C> {
570            fn apply(self, state: &mut SinkState<SourceData, Timestamp>) {
571                debug!("ct_sink event {:?}", self);
572                match self {
573                    OpEvent::ToAppend(Event::Data(_cap, x)) => {
574                        for (k, t, d) in x {
575                            state.to_append.push(((k, t), d));
576                        }
577                    }
578                    OpEvent::ToAppend(Event::Progress(x)) => state.to_append_progress = x,
579                    OpEvent::AppendTimes(Event::Data(_cap, x)) => state
580                        .append_times
581                        .extend(x.into_iter().map(|((), t, _d)| t)),
582                    OpEvent::AppendTimes(Event::Progress(x)) => state.append_times_progress = x,
583                }
584            }
585        }
586
587        let to_insert_input = to_append_input.map(OpEvent::ToAppend);
588        let append_times_input = append_times_input.map(OpEvent::AppendTimes);
589        let mut op_inputs = futures::stream::select(to_insert_input, append_times_input);
590
591        let mut state = SinkState::new();
592        loop {
593            // Loop until we've processed all the work we can.
594            loop {
595                if PartialOrder::less_than(&*output_frontier.borrow(), &state.output_progress) {
596                    output_frontier.borrow_mut().clear();
597                    output_frontier
598                        .borrow_mut()
599                        .extend(state.output_progress.iter().cloned());
600                }
601
602                debug!("ct_sink about to process {:?}", state);
603                let Some((new_upper, to_append)) = state.process() else {
604                    break;
605                };
606                debug!("ct_sink got write {:?}: {:?}", new_upper, to_append);
607                state.output_progress =
608                    truncating_compare_and_append(&mut write_handle, to_append, new_upper).await;
609            }
610
611            // Then try to generate some more work by reading inputs.
612            let Some(event) = op_inputs.next().await else {
613                // Inputs exhausted, shutting down.
614                output_frontier.borrow_mut().clear();
615                return;
616            };
617            event.apply(&mut state);
618            // Also drain any other events that may be ready.
619            while let Some(Some(event)) = op_inputs.next().now_or_never() {
620                event.apply(&mut state);
621            }
622        }
623    });
624
625    button
626}
627
628/// Writes the given data to the shard, truncating it as necessary.
629///
630/// Returns the latest known upper for the shard.
631async fn truncating_compare_and_append(
632    write_handle: &mut WriteHandle<SourceData, (), Timestamp, StorageDiff>,
633    to_append: Vec<((&SourceData, &()), &Timestamp, StorageDiff)>,
634    new_upper: Antichain<Timestamp>,
635) -> Antichain<Timestamp> {
636    let mut expected_upper = write_handle.shared_upper();
637    loop {
638        if !PartialOrder::less_than(&expected_upper, &new_upper) {
639            debug!("ct_sink skipping {:?}", new_upper.elements());
640            return expected_upper;
641        }
642        let res = write_handle
643            .compare_and_append(&to_append, expected_upper.clone(), new_upper.clone())
644            .await
645            .expect("usage was valid");
646        debug!(
647            "ct_sink write res {:?}-{:?}: {:?}",
648            expected_upper.elements(),
649            new_upper.elements(),
650            res
651        );
652        match res {
653            Ok(()) => return new_upper,
654            Err(err) => {
655                expected_upper = err.current;
656                continue;
657            }
658        }
659    }
660}
661
662#[derive(Debug)]
663struct SinkState<D, T> {
664    /// The known times at which we're going to write data to the output. This
665    /// is guaranteed to include all times < append_times_progress, except that
666    /// ones < output_progress may have been truncated.
667    append_times: BTreeSet<T>,
668    append_times_progress: Antichain<T>,
669
670    /// The data we've collected to append to the output. This is often
671    /// compacted to advancing times and is expected to be ~empty in the steady
672    /// state.
673    to_append: ConsolidatingVec<(D, T)>,
674    to_append_progress: Antichain<T>,
675
676    /// A lower bound on the upper of the output.
677    output_progress: Antichain<T>,
678}
679
680impl<D: Ord> SinkState<D, Timestamp> {
681    fn new() -> Self {
682        SinkState {
683            append_times: BTreeSet::new(),
684            append_times_progress: Antichain::from_elem(Timestamp::minimum()),
685            to_append: ConsolidatingVec::new(128, 0),
686            to_append_progress: Antichain::from_elem(Timestamp::minimum()),
687            output_progress: Antichain::from_elem(Timestamp::minimum()),
688        }
689    }
690
691    /// Returns data to write to the output, if any, and the new upper to use.
692    fn process(
693        &mut self,
694    ) -> Option<(
695        Antichain<Timestamp>,
696        Vec<((&D, &()), &Timestamp, StorageDiff)>,
697    )> {
698        // We can only append at times >= the output_progress, so pop off
699        // anything unnecessary.
700        while let Some(x) = self.append_times.first() {
701            if self.output_progress.less_equal(x) {
702                break;
703            }
704            self.append_times.pop_first();
705        }
706
707        // Find the smallest append_time before append_time_progress. This is
708        // the next time we might need to write data at. Note that we can only
709        // act on append_times once the progress has passed them, because they
710        // could come out of order.
711        let write_ts = match self.append_times.first() {
712            Some(x) if !self.append_times_progress.less_equal(x) => x,
713            Some(_) | None => {
714                // The CT sink's contract is that it only writes data at times
715                // we received an input diff. There are none in
716                // `[output_progress, append_times_progress)`, so we can go
717                // ahead and advance the upper of the output, if it's not
718                // already.
719                //
720                // We could instead ensure liveness by basing this off of
721                // to_append, but for any CTs reading the output (expected to be
722                // a common case) we'd end up looping each timestamp through
723                // persist one-by-one.
724                if PartialOrder::less_than(&self.output_progress, &self.append_times_progress) {
725                    return Some((self.append_times_progress.clone(), Vec::new()));
726                }
727                // Otherwise, nothing to do!
728                return None;
729            }
730        };
731
732        if self.to_append_progress.less_equal(write_ts) {
733            // Don't have all the necessary data yet.
734            if self.output_progress.less_than(write_ts) {
735                // We can advance the output upper up to the write_ts. For
736                // self-referential CTs this might be necessary to ensure
737                // dataflow progress.
738                return Some((Antichain::from_elem(write_ts.clone()), Vec::new()));
739            }
740            return None;
741        }
742
743        // Time to write some data! Produce the collection as of write_ts by
744        // advancing timestamps, consolidating, and filtering out anything at
745        // future timestamps.
746        let as_of = std::slice::from_ref(write_ts);
747        for ((_, t), _) in self.to_append.iter_mut() {
748            t.advance_by(AntichainRef::new(as_of))
749        }
750        // TODO(ct2): Metrics for vec len and cap.
751        self.to_append.consolidate();
752
753        let append_data = self
754            .to_append
755            .iter()
756            .filter_map(|((k, t), d)| (t <= write_ts).then_some(((k, &()), t, d.into_inner())))
757            .collect();
758        Some((Antichain::from_elem(write_ts.step_forward()), append_data))
759    }
760}
761
762trait StepForward<G: Scope, D, R> {
763    /// Translates a collection one timestamp "forward" (i.e. `T` -> `T+1` as
764    /// defined by `TimestampManipulation::step_forward`).
765    ///
766    /// This includes:
767    /// - The differential timestamps in each data.
768    /// - The capabilities paired with that data.
769    /// - (As a consequence of the previous) the output frontier is one step forward
770    ///   of the input frontier.
771    ///
772    /// The caller is responsible for ensuring that all data and capabilities given
773    /// to this operator can be stepped forward without panicking, otherwise the
774    /// operator will panic at runtime.
775    fn step_forward(&self, name: &str) -> VecCollection<G, D, R>;
776}
777
778impl<G, D, R> StepForward<G, D, R> for VecCollection<G, D, R>
779where
780    G: Scope<Timestamp = Timestamp>,
781    D: Data,
782    R: Semigroup + 'static,
783{
784    fn step_forward(&self, name: &str) -> VecCollection<G, D, R> {
785        let name = format!("ct_step_forward({})", name);
786        let mut builder = OperatorBuilder::new(name, self.scope());
787        let (output, output_stream) = builder.new_output();
788        let mut output = OutputBuilder::from(output);
789
790        // We step forward (by one) each data timestamp and capability. As a
791        // result the output's frontier is guaranteed to be one past the input
792        // frontier, so make this promise to timely.
793        let step_forward_summary = Timestamp::from(1);
794        let mut input = builder.new_input_connection(
795            &self.inner,
796            Pipeline,
797            [(0, Antichain::from_elem(step_forward_summary))],
798        );
799        builder.set_notify(false);
800        builder.build(move |_caps| {
801            move |_frontiers| {
802                let mut output = output.activate();
803                while let Some((cap, data)) = input.next() {
804                    for (_, ts, _) in data.iter_mut() {
805                        *ts = ts.step_forward();
806                    }
807                    let cap = cap.delayed(&cap.time().step_forward());
808                    output.session(&cap).give_container(data);
809                }
810            }
811        });
812
813        output_stream.as_collection()
814    }
815}
816
817trait TimesExtract<G: Scope, D, R> {
818    /// Returns a collection with the times changed in the input collection.
819    ///
820    /// This works by mapping the data piece of the differential tuple to `()`.
821    /// It is essentially the same as the following, but without cloning
822    /// everything in the input.
823    ///
824    /// ```ignore
825    /// input.map(|(_data, ts, diff)| ((), ts, diff))
826    /// ```
827    ///
828    /// The output may be partially consolidated, but no consolidation
829    /// guarantees are made.
830    fn times_extract(&self, name: &str) -> (VecCollection<G, D, R>, VecCollection<G, (), R>);
831}
832
833impl<G, D, R> TimesExtract<G, D, R> for VecCollection<G, D, R>
834where
835    G: Scope<Timestamp = Timestamp>,
836    D: Clone + 'static,
837    R: Semigroup + 'static + std::fmt::Debug,
838{
839    fn times_extract(&self, name: &str) -> (VecCollection<G, D, R>, VecCollection<G, (), R>) {
840        let name = format!("ct_times_extract({})", name);
841        let mut builder = OperatorBuilder::new(name, self.scope());
842        let (passthrough, passthrough_stream) = builder.new_output();
843        let mut passthrough = OutputBuilder::from(passthrough);
844        let (times, times_stream) = builder.new_output();
845        let mut times = OutputBuilder::<_, ConsolidatingContainerBuilder<_>>::from(times);
846        let mut input = builder.new_input(&self.inner, Pipeline);
847        builder.set_notify(false);
848        builder.build(|_caps| {
849            move |_frontiers| {
850                let mut passthrough = passthrough.activate();
851                let mut times = times.activate();
852                input.for_each_time(|time, data| {
853                    let mut times_session = times.session_with_builder(&time);
854                    let mut passthrough_session = passthrough.session(&time);
855                    for data in data {
856                        let times_iter =
857                            data.iter().map(|(_data, ts, diff)| ((), *ts, diff.clone()));
858                        times_session.give_iterator(times_iter);
859                        passthrough_session.give_container(data);
860                    }
861                });
862            }
863        });
864        (
865            passthrough_stream.as_collection(),
866            times_stream.as_collection(),
867        )
868    }
869}
870
871trait TimesReduce<G: Scope, R> {
872    /// This is essentially a specialized impl of consolidate, with a HashMap
873    /// instead of the Trace.
874    fn times_reduce(&self, name: &str) -> VecCollection<G, (), R>;
875}
876
877impl<G, R> TimesReduce<G, R> for VecCollection<G, (), R>
878where
879    G: Scope<Timestamp = Timestamp>,
880    R: Semigroup + 'static + std::fmt::Debug,
881{
882    fn times_reduce(&self, name: &str) -> VecCollection<G, (), R> {
883        let name = format!("ct_times_reduce({})", name);
884        self.inner
885            .unary_frontier(Pipeline, &name, |_caps, _info| {
886                let mut notificator = FrontierNotificator::default();
887                let mut stash = HashMap::<_, R>::new();
888                move |(input, frontier), output| {
889                    input.for_each(|cap, data| {
890                        for ((), ts, diff) in data.drain(..) {
891                            notificator.notify_at(cap.delayed(&ts));
892                            if let Some(sum) = stash.get_mut(&ts) {
893                                sum.plus_equals(&diff);
894                            } else {
895                                stash.insert(ts, diff);
896                            }
897                        }
898                    });
899                    notificator.for_each(&[frontier], |cap, _not| {
900                        if let Some(diff) = stash.remove(cap.time()) {
901                            output.session(&cap).give(((), cap.time().clone(), diff));
902                        }
903                    });
904                }
905            })
906            .as_collection()
907    }
908}
909
910#[cfg(test)]
911mod tests {
912    use differential_dataflow::AsCollection;
913    use mz_repr::Timestamp;
914    use timely::Config;
915    use timely::dataflow::ProbeHandle;
916    use timely::dataflow::operators::capture::Extract;
917    use timely::dataflow::operators::{Capture, Input, ToStream};
918    use timely::progress::Antichain;
919
920    use super::*;
921
922    #[mz_ore::test]
923    fn step_forward() {
924        timely::execute(Config::thread(), |worker| {
925            let (mut input, probe, output) = worker.dataflow(|scope| {
926                let (handle, input) = scope.new_input();
927                let probe = ProbeHandle::<Timestamp>::new();
928                let output = input
929                    .as_collection()
930                    .step_forward("test")
931                    .probe_with(&probe)
932                    .inner
933                    .capture();
934                (handle, probe, output)
935            });
936
937            let mut expected = Vec::new();
938            for i in 0u64..10 {
939                let in_ts = Timestamp::new(i);
940                let out_ts = in_ts.step_forward();
941                input.send((i, in_ts, 1));
942                input.advance_to(in_ts.step_forward());
943
944                // We should get the data out advanced by `step_forward` and
945                // also, crucially, the output frontier should do the same (i.e.
946                // this is why we can't simply use `VecCollection::delay`).
947                worker.step_while(|| probe.less_than(&out_ts.step_forward()));
948                expected.push((i, out_ts, 1));
949            }
950            // Closing the input should allow the output to advance and the
951            // dataflow to shut down.
952            input.close();
953            while worker.step() {}
954
955            let actual = output
956                .extract()
957                .into_iter()
958                .flat_map(|x| x.1)
959                .collect::<Vec<_>>();
960            assert_eq!(actual, expected);
961        })
962        .unwrap();
963    }
964
965    #[mz_ore::test]
966    fn times_extract() {
967        struct PanicOnClone;
968
969        impl Clone for PanicOnClone {
970            fn clone(&self) -> Self {
971                panic!("boom")
972            }
973        }
974
975        let output = timely::execute_directly(|worker| {
976            worker.dataflow(|scope| {
977                let input = [
978                    (PanicOnClone, Timestamp::new(0), 0),
979                    (PanicOnClone, Timestamp::new(1), 1),
980                    (PanicOnClone, Timestamp::new(1), 1),
981                    (PanicOnClone, Timestamp::new(2), -2),
982                    (PanicOnClone, Timestamp::new(2), 1),
983                ]
984                .to_stream(scope)
985                .as_collection();
986                let (_passthrough, times) = input.times_extract("test");
987                times.inner.capture()
988            })
989        });
990        let expected = vec![((), Timestamp::new(1), 2), ((), Timestamp::new(2), -1)];
991        let actual = output
992            .extract()
993            .into_iter()
994            .flat_map(|x| x.1)
995            .collect::<Vec<_>>();
996        assert_eq!(actual, expected);
997    }
998
999    #[mz_ore::test]
1000    fn times_reduce() {
1001        let output = timely::execute_directly(|worker| {
1002            worker.dataflow(|scope| {
1003                let input = [
1004                    ((), Timestamp::new(3), 1),
1005                    ((), Timestamp::new(2), 1),
1006                    ((), Timestamp::new(1), 1),
1007                    ((), Timestamp::new(2), 1),
1008                    ((), Timestamp::new(3), 1),
1009                    ((), Timestamp::new(3), 1),
1010                ]
1011                .to_stream(scope)
1012                .as_collection();
1013                input.times_reduce("test").inner.capture()
1014            })
1015        });
1016        let expected = vec![
1017            ((), Timestamp::new(1), 1),
1018            ((), Timestamp::new(2), 2),
1019            ((), Timestamp::new(3), 3),
1020        ];
1021        let actual = output
1022            .extract()
1023            .into_iter()
1024            .flat_map(|x| x.1)
1025            .collect::<Vec<_>>();
1026        assert_eq!(actual, expected);
1027    }
1028
1029    #[mz_ore::test]
1030    fn ct_sink_state() {
1031        #[track_caller]
1032        fn assert_noop(state: &mut super::SinkState<&'static str, Timestamp>) {
1033            if let Some(ret) = state.process() {
1034                panic!("should be nothing to write: {:?}", ret);
1035            }
1036        }
1037
1038        #[track_caller]
1039        fn assert_write(
1040            state: &mut super::SinkState<&'static str, Timestamp>,
1041            expected_upper: u64,
1042            expected_append: &[&str],
1043        ) {
1044            let (new_upper, to_append) = state.process().expect("should be something to write");
1045            assert_eq!(
1046                new_upper,
1047                Antichain::from_elem(Timestamp::new(expected_upper))
1048            );
1049            let to_append = to_append
1050                .into_iter()
1051                .map(|((k, ()), _ts, _diff)| *k)
1052                .collect::<Vec<_>>();
1053            assert_eq!(to_append, expected_append);
1054        }
1055
1056        let mut s = super::SinkState::new();
1057
1058        // Nothing to do at the initial state.
1059        assert_noop(&mut s);
1060
1061        // Getting data to append is not enough to do anything yet.
1062        s.to_append.push((("a", 1.into()), Diff::ONE));
1063        s.to_append.push((("b", 1.into()), Diff::ONE));
1064        assert_noop(&mut s);
1065
1066        // Knowing that this is the only data we'll get for that timestamp is
1067        // still not enough.
1068        s.to_append_progress = Antichain::from_elem(2.into());
1069        assert_noop(&mut s);
1070
1071        // Even knowing that we got input at that time is not quite enough yet
1072        // (we could be getting these out of order).
1073        s.append_times.insert(1.into());
1074        assert_noop(&mut s);
1075
1076        // Indeed, it did come out of order. Also note that this checks the ==
1077        // case for time vs progress.
1078        s.append_times.insert(0.into());
1079        assert_noop(&mut s);
1080
1081        // Okay, now we know that we've seen all the times we got input up to 3.
1082        // This is enough to allow the empty write of `[0,1)`.
1083        s.append_times_progress = Antichain::from_elem(3.into());
1084        assert_write(&mut s, 1, &[]);
1085
1086        // That succeeded, now we can write the data at 1.
1087        s.output_progress = Antichain::from_elem(1.into());
1088        assert_write(&mut s, 2, &["a", "b"]);
1089
1090        // That succeeded, now we know about some empty time.
1091        s.output_progress = Antichain::from_elem(2.into());
1092        assert_write(&mut s, 3, &[]);
1093
1094        // That succeeded, now nothing to do.
1095        s.output_progress = Antichain::from_elem(3.into());
1096        assert_noop(&mut s);
1097
1098        // Find out about a new time to write at. Even without the data, we can
1099        // do an empty write up to that time.
1100        s.append_times.insert(5.into());
1101        s.append_times_progress = Antichain::from_elem(6.into());
1102        assert_write(&mut s, 5, &[]);
1103
1104        // That succeeded, now nothing to do again.
1105        s.output_progress = Antichain::from_elem(5.into());
1106
1107        // Retract one of the things currently in the collection and add a new
1108        // thing, to verify the consolidate.
1109        s.to_append.push((("a", 5.into()), Diff::MINUS_ONE));
1110        s.to_append.push((("c", 5.into()), Diff::ONE));
1111        s.to_append_progress = Antichain::from_elem(6.into());
1112        assert_write(&mut s, 6, &["b", "c"]);
1113    }
1114}