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