1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! Coordinator bookkeeping for active compute sinks.

use std::cmp::Ordering;
use std::collections::BTreeSet;
use std::iter;

use anyhow::anyhow;
use itertools::Itertools;
use mz_adapter_types::connection::ConnectionId;
use mz_compute_client::protocol::response::SubscribeBatch;
use mz_controller_types::ClusterId;
use mz_expr::compare_columns;
use mz_ore::cast::CastFrom;
use mz_ore::now::EpochMillis;
use mz_repr::adt::numeric;
use mz_repr::{Datum, GlobalId, IntoRowIterator, Row, Timestamp};
use mz_sql::plan::SubscribeOutput;
use timely::progress::Antichain;
use tokio::sync::{mpsc, oneshot};
use uuid::Uuid;

use crate::coord::peek::PeekResponseUnary;
use crate::{AdapterError, ExecuteResponse};

#[derive(Debug)]
/// A description of an active compute sink from the coordinator's perspective.
pub enum ActiveComputeSink {
    /// An active subscribe sink.
    Subscribe(ActiveSubscribe),
    /// An active copy to sink.
    CopyTo(ActiveCopyTo),
}

impl ActiveComputeSink {
    /// Reports the ID of the cluster on which the sink is running.
    pub fn cluster_id(&self) -> ClusterId {
        match &self {
            ActiveComputeSink::Subscribe(subscribe) => subscribe.cluster_id,
            ActiveComputeSink::CopyTo(copy_to) => copy_to.cluster_id,
        }
    }

    /// Reports the ID of the connection which created the sink.
    pub fn connection_id(&self) -> &ConnectionId {
        match &self {
            ActiveComputeSink::Subscribe(subscribe) => &subscribe.conn_id,
            ActiveComputeSink::CopyTo(copy_to) => &copy_to.conn_id,
        }
    }

    /// Reports the IDs of the objects on which the sink depends.
    pub fn depends_on(&self) -> &BTreeSet<GlobalId> {
        match &self {
            ActiveComputeSink::Subscribe(subscribe) => &subscribe.depends_on,
            ActiveComputeSink::CopyTo(copy_to) => &copy_to.depends_on,
        }
    }

    /// Retires the sink with the specified reason.
    ///
    /// This method must be called on every sink before it is dropped. It
    /// informs the end client that the sink is finished for the specified
    /// reason.
    pub fn retire(self, reason: ActiveComputeSinkRetireReason) {
        match self {
            ActiveComputeSink::Subscribe(subscribe) => subscribe.retire(reason),
            ActiveComputeSink::CopyTo(copy_to) => copy_to.retire(reason),
        }
    }
}

/// The reason for removing an [`ActiveComputeSink`].
#[derive(Debug, Clone)]
pub enum ActiveComputeSinkRetireReason {
    /// The compute sink completed successfully.
    Finished,
    /// The compute sink was canceled due to a user request.
    Canceled,
    /// The compute sink was forcibly terminated because an object it depended on
    /// was dropped.
    DependencyDropped(String),
}

/// A description of an active subscribe from coord's perspective
#[derive(Debug)]
pub struct ActiveSubscribe {
    /// The ID of the connection which created the subscribe.
    pub conn_id: ConnectionId,
    /// The UUID of the session which created the subscribe.
    pub session_uuid: Uuid,
    /// The ID of the cluster on which the subscribe is running.
    pub cluster_id: ClusterId,
    /// The IDs of the objects on which the subscribe depends.
    pub depends_on: BTreeSet<GlobalId>,
    /// Channel on which to send responses to the client.
    // The responses have the form `PeekResponseUnary` but should perhaps
    // become `SubscribeResponse`.
    pub channel: mpsc::UnboundedSender<PeekResponseUnary>,
    /// Whether progress information should be emitted.
    pub emit_progress: bool,
    /// The logical timestamp at which the subscribe began execution.
    pub as_of: Timestamp,
    /// The number of columns in the relation that was subscribed to.
    pub arity: usize,
    /// The time when the subscribe started.
    pub start_time: EpochMillis,
    /// How to present the subscribe's output.
    pub output: SubscribeOutput,
}

impl ActiveSubscribe {
    /// Initializes the subscription.
    ///
    /// This method must be called exactly once, after constructing an
    /// `ActiveSubscribe` and before calling `process_response`.
    pub fn initialize(&self) {
        // Always emit progress message indicating snapshot timestamp.
        self.send_progress_message(&Antichain::from_elem(self.as_of));
    }

    fn send_progress_message(&self, upper: &Antichain<Timestamp>) {
        if !self.emit_progress {
            return;
        }
        if let Some(upper) = upper.as_option() {
            let mut row_buf = Row::default();
            let mut packer = row_buf.packer();
            packer.push(Datum::from(numeric::Numeric::from(*upper)));
            packer.push(Datum::True);

            // Fill in the mz_diff or mz_state column
            packer.push(Datum::Null);

            // Fill all table columns with NULL.
            for _ in 0..self.arity {
                packer.push(Datum::Null);
            }

            if let SubscribeOutput::EnvelopeDebezium { order_by_keys } = &self.output {
                for _ in 0..(self.arity - order_by_keys.len()) {
                    packer.push(Datum::Null);
                }
            }

            let row_iter = Box::new(row_buf.into_row_iter());
            self.send(PeekResponseUnary::Rows(row_iter));
        }
    }

    /// Processes a subscribe response from the controller.
    ///
    /// Returns `true` if the subscribe is finished.
    pub fn process_response(&self, batch: SubscribeBatch) -> bool {
        let mut rows = match batch.updates {
            Ok(rows) => rows,
            Err(s) => {
                self.send(PeekResponseUnary::Error(s));
                return true;
            }
        };

        // Sort results by time. We use stable sort here because it will produce
        // deterministic results since the cursor will always produce rows in
        // the same order. Compute doesn't guarantee that the results are sorted
        // (materialize#18936)
        let mut row_buf = Row::default();
        match &self.output {
            SubscribeOutput::WithinTimestampOrderBy { order_by } => {
                let mut left_datum_vec = mz_repr::DatumVec::new();
                let mut right_datum_vec = mz_repr::DatumVec::new();
                rows.sort_by(
                    |(left_time, left_row, left_diff), (right_time, right_row, right_diff)| {
                        left_time.cmp(right_time).then_with(|| {
                            let mut left_datums = left_datum_vec.borrow();
                            left_datums.extend(&[Datum::Int64(*left_diff)]);
                            left_datums.extend(left_row.iter());
                            let mut right_datums = right_datum_vec.borrow();
                            right_datums.extend(&[Datum::Int64(*right_diff)]);
                            right_datums.extend(right_row.iter());
                            compare_columns(order_by, &left_datums, &right_datums, || {
                                left_row.cmp(right_row).then(left_diff.cmp(right_diff))
                            })
                        })
                    },
                );
            }
            SubscribeOutput::EnvelopeUpsert { order_by_keys }
            | SubscribeOutput::EnvelopeDebezium { order_by_keys } => {
                let debezium = matches!(self.output, SubscribeOutput::EnvelopeDebezium { .. });
                let mut left_datum_vec = mz_repr::DatumVec::new();
                let mut right_datum_vec = mz_repr::DatumVec::new();
                rows.sort_by(
                    |(left_time, left_row, left_diff), (right_time, right_row, right_diff)| {
                        left_time.cmp(right_time).then_with(|| {
                            let left_datums = left_datum_vec.borrow_with(left_row);
                            let right_datums = right_datum_vec.borrow_with(right_row);
                            compare_columns(order_by_keys, &left_datums, &right_datums, || {
                                left_diff.cmp(right_diff)
                            })
                        })
                    },
                );

                let mut new_rows = Vec::new();
                let mut it = rows.iter();
                let mut datum_vec = mz_repr::DatumVec::new();
                let mut old_datum_vec = mz_repr::DatumVec::new();
                while let Some(start) = it.next() {
                    let group = iter::once(start)
                        .chain(it.take_while_ref(|row| {
                            let left_datums = left_datum_vec.borrow_with(&start.1);
                            let right_datums = right_datum_vec.borrow_with(&row.1);
                            start.0 == row.0
                                && compare_columns(
                                    order_by_keys,
                                    &left_datums,
                                    &right_datums,
                                    || Ordering::Equal,
                                ) == Ordering::Equal
                        }))
                        .collect_vec();

                    // Four cases:
                    // [(key, value, +1)] => ("insert", key, NULL, value)
                    // [(key, v1, -1), (key, v2, +1)] => ("upsert", key, v1, v2)
                    // [(key, value, -1)] => ("delete", key, value, NULL)
                    // everything else => ("key_violation", key, NULL, NULL)
                    let value_columns = self.arity - order_by_keys.len();
                    let mut packer = row_buf.packer();
                    new_rows.push(match &group[..] {
                        [(_, row, 1)] => {
                            packer.push(if debezium {
                                Datum::String("insert")
                            } else {
                                Datum::String("upsert")
                            });
                            let datums = datum_vec.borrow_with(row);
                            for column_order in order_by_keys {
                                packer.push(datums[column_order.column]);
                            }
                            if debezium {
                                for _ in 0..value_columns {
                                    packer.push(Datum::Null);
                                }
                            }
                            for idx in 0..self.arity {
                                if !order_by_keys.iter().any(|co| co.column == idx) {
                                    packer.push(datums[idx]);
                                }
                            }
                            (start.0, row_buf.clone(), 0)
                        }
                        [(_, _, -1)] => {
                            packer.push(Datum::String("delete"));
                            let datums = datum_vec.borrow_with(&start.1);
                            for column_order in order_by_keys {
                                packer.push(datums[column_order.column]);
                            }
                            if debezium {
                                for idx in 0..self.arity {
                                    if !order_by_keys.iter().any(|co| co.column == idx) {
                                        packer.push(datums[idx]);
                                    }
                                }
                            }
                            for _ in 0..self.arity - order_by_keys.len() {
                                packer.push(Datum::Null);
                            }
                            (start.0, row_buf.clone(), 0)
                        }
                        [(_, old_row, -1), (_, row, 1)] => {
                            packer.push(Datum::String("upsert"));
                            let datums = datum_vec.borrow_with(row);
                            let old_datums = old_datum_vec.borrow_with(old_row);

                            for column_order in order_by_keys {
                                packer.push(datums[column_order.column]);
                            }
                            if debezium {
                                for idx in 0..self.arity {
                                    if !order_by_keys.iter().any(|co| co.column == idx) {
                                        packer.push(old_datums[idx]);
                                    }
                                }
                            }
                            for idx in 0..self.arity {
                                if !order_by_keys.iter().any(|co| co.column == idx) {
                                    packer.push(datums[idx]);
                                }
                            }
                            (start.0, row_buf.clone(), 0)
                        }
                        _ => {
                            packer.push(Datum::String("key_violation"));
                            let datums = datum_vec.borrow_with(&start.1);
                            for column_order in order_by_keys {
                                packer.push(datums[column_order.column]);
                            }
                            if debezium {
                                for _ in 0..(self.arity - order_by_keys.len()) {
                                    packer.push(Datum::Null);
                                }
                            }
                            for _ in 0..(self.arity - order_by_keys.len()) {
                                packer.push(Datum::Null);
                            }
                            (start.0, row_buf.clone(), 0)
                        }
                    });
                }
                rows = new_rows;
            }
            SubscribeOutput::Diffs => rows.sort_by_key(|(time, _, _)| *time),
        }

        let rows: Vec<Row> = rows
            .into_iter()
            .map(|(time, row, diff)| {
                assert!(self.as_of <= time);
                let mut packer = row_buf.packer();
                // TODO: Change to MzTimestamp.
                packer.push(Datum::from(numeric::Numeric::from(time)));
                if self.emit_progress {
                    // When sinking with PROGRESS, the output includes an
                    // additional column that indicates whether a timestamp is
                    // complete. For regular "data" updates this is always
                    // `false`.
                    packer.push(Datum::False);
                }

                match &self.output {
                    SubscribeOutput::EnvelopeUpsert { .. }
                    | SubscribeOutput::EnvelopeDebezium { .. } => {}
                    SubscribeOutput::Diffs | SubscribeOutput::WithinTimestampOrderBy { .. } => {
                        packer.push(Datum::Int64(diff));
                    }
                }

                packer.extend_by_row(&row);

                row_buf.clone()
            })
            .collect();
        let rows = Box::new(rows.into_row_iter());

        self.send(PeekResponseUnary::Rows(rows));

        // Emit progress message if requested. Don't emit progress for the first
        // batch if the upper is exactly `as_of` (we're guaranteed it is not
        // less than `as_of`, but it might be exactly `as_of`) as we've already
        // emitted that progress message in `initialize`.
        if !batch.upper.less_equal(&self.as_of) {
            self.send_progress_message(&batch.upper);
        }

        batch.upper.is_empty()
    }

    /// Retires the subscribe with the specified reason.
    ///
    /// This method must be called on every subscribe before it is dropped. It
    /// informs the end client that the subscribe is finished for the specified
    /// reason.
    pub fn retire(self, reason: ActiveComputeSinkRetireReason) {
        let message = match reason {
            ActiveComputeSinkRetireReason::Finished => return,
            ActiveComputeSinkRetireReason::Canceled => PeekResponseUnary::Canceled,
            ActiveComputeSinkRetireReason::DependencyDropped(d) => PeekResponseUnary::Error(
                format!("subscribe has been terminated because underlying {d} was dropped"),
            ),
        };
        self.send(message);
    }

    /// Sends a message to the client if the subscribe has not already completed
    /// and if the client has not already gone away.
    fn send(&self, response: PeekResponseUnary) {
        // TODO(benesch): the lack of backpressure here can result in
        // unbounded memory usage.
        let _ = self.channel.send(response);
    }
}

/// A description of an active copy to sink from the coordinator's perspective.
#[derive(Debug)]
pub struct ActiveCopyTo {
    /// The ID of the connection which created the subscribe.
    pub conn_id: ConnectionId,
    /// The result channel for the `COPY ... TO` statement that created the copy to sink.
    pub tx: oneshot::Sender<Result<ExecuteResponse, AdapterError>>,
    /// The ID of the cluster on which the copy to is running.
    pub cluster_id: ClusterId,
    /// The IDs of the objects on which the copy to depends.
    pub depends_on: BTreeSet<GlobalId>,
}

impl ActiveCopyTo {
    /// Retires the copy to with a response from the controller.
    ///
    /// Unlike subscribes, copy tos only expect a single response from the
    /// controller, so `process_response` and `retire` are unified into a single
    /// operation.
    ///
    /// Either this method or `retire` must be called on every copy to before it
    /// is dropped.
    pub fn retire_with_response(self, response: Result<u64, anyhow::Error>) {
        let response = match response {
            Ok(n) => Ok(ExecuteResponse::Copied(usize::cast_from(n))),
            Err(error) => Err(AdapterError::Unstructured(error)),
        };
        let _ = self.tx.send(response);
    }

    /// Retires the copy to with the specified reason.
    ///
    /// Either this method or `retire_with_response` must be called on every
    /// copy to before it is dropped.
    pub fn retire(self, reason: ActiveComputeSinkRetireReason) {
        let message = match reason {
            ActiveComputeSinkRetireReason::Finished => return,
            ActiveComputeSinkRetireReason::Canceled => Err(AdapterError::Canceled),
            ActiveComputeSinkRetireReason::DependencyDropped(d) => Err(AdapterError::Unstructured(
                anyhow!("copy has been terminated because underlying {d} was dropped"),
            )),
        };
        let _ = self.tx.send(message);
    }
}