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
// 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.

//! Logic related to the creation of dataflow sinks.

use std::cell::RefCell;
use std::rc::Rc;
use std::sync::Arc;

use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::trace::implementations::ord_neu::ColValSpine;
use differential_dataflow::{AsCollection, Collection, Hashable};
use mz_interchange::envelopes::{combine_at_timestamp, dbz_format};
use mz_persist_client::operators::shard_source::SnapshotMode;
use mz_repr::{Datum, Diff, GlobalId, Row, Timestamp};
use mz_storage_operators::persist_source;
use mz_storage_types::errors::DataflowError;
use mz_storage_types::sinks::{
    MetadataFilled, SinkEnvelope, StorageSinkConnection, StorageSinkDesc,
};
use mz_timely_util::builder_async::PressOnDropButton;
use timely::dataflow::operators::Leave;
use timely::dataflow::scopes::Child;
use timely::dataflow::{Scope, Stream};
use tracing::warn;

use crate::healthcheck::HealthStatusMessage;
use crate::storage_state::StorageState;

/// _Renders_ complete _differential_ [`Collection`]s
/// that represent the sink and its errors as requested
/// by the original `CREATE SINK` statement.
pub(crate) fn render_sink<'g, G: Scope<Timestamp = ()>>(
    scope: &mut Child<'g, G, mz_repr::Timestamp>,
    storage_state: &mut StorageState,
    sink_id: GlobalId,
    sink: &StorageSinkDesc<MetadataFilled, mz_repr::Timestamp>,
) -> (Stream<G, HealthStatusMessage>, Vec<PressOnDropButton>) {
    let sink_render = get_sink_render_for(&sink.connection);

    let mut tokens = vec![];

    let snapshot_mode = if sink.with_snapshot {
        SnapshotMode::Include
    } else {
        SnapshotMode::Exclude
    };
    let (ok_collection, err_collection, persist_tokens) = persist_source::persist_source(
        scope,
        sink.from,
        Arc::clone(&storage_state.persist_clients),
        sink.from_storage_metadata.clone(),
        Some(sink.as_of.clone()),
        snapshot_mode,
        timely::progress::Antichain::new(),
        None,
        None,
        async {},
    );
    tokens.extend(persist_tokens);

    let ok_collection =
        apply_sink_envelope(sink_id, sink, &sink_render, ok_collection.as_collection());

    let (health, sink_tokens) = sink_render.render_sink(
        storage_state,
        sink,
        sink_id,
        ok_collection,
        err_collection.as_collection(),
    );

    tokens.extend(sink_tokens);

    (health.leave(), tokens)
}

#[allow(clippy::borrowed_box)]
fn apply_sink_envelope<G>(
    sink_id: GlobalId,
    sink: &StorageSinkDesc<MetadataFilled, mz_repr::Timestamp>,
    sink_render: &Box<dyn SinkRender<G>>,
    collection: Collection<G, Row, Diff>,
) -> Collection<G, (Option<Row>, Option<Row>), Diff>
where
    G: Scope<Timestamp = Timestamp>,
{
    // Some connections support keys - extract them.
    let keyed = if sink_render.uses_keys() {
        let user_key_indices = sink_render
            .get_key_indices()
            .map(|key_indices| key_indices.to_vec());

        let relation_key_indices = sink_render
            .get_relation_key_indices()
            .map(|key_indices| key_indices.to_vec());

        // We have three cases here, in descending priority:
        //
        // 1. if there is a user-specified key, use that to consolidate and
        //  distribute work
        // 2. if the sinked relation has a known primary key, use that to
        //  consolidate and distribute work but don't write to the sink
        // 3. if none of the above, use the whole row as key to
        //  consolidate and distribute work but don't write to the sink

        let keyed = if let Some(key_indices) = user_key_indices {
            let mut datum_vec = mz_repr::DatumVec::new();
            collection.map(move |row| {
                // TODO[perf] (btv) - is there a way to avoid unpacking and repacking every row and cloning the datums?
                // Does it matter?
                let key = {
                    let datums = datum_vec.borrow_with(&row);
                    Row::pack(key_indices.iter().map(|&idx| datums[idx].clone()))
                };
                (Some(key), row)
            })
        } else if let Some(relation_key_indices) = relation_key_indices {
            let mut datum_vec = mz_repr::DatumVec::new();
            collection.map(move |row| {
                // TODO[perf] (btv) - is there a way to avoid unpacking and repacking every row and cloning the datums?
                // Does it matter?
                let key = {
                    let datums = datum_vec.borrow_with(&row);
                    Row::pack(relation_key_indices.iter().map(|&idx| datums[idx].clone()))
                };
                (Some(key), row)
            })
        } else {
            collection.map(|row| (Some(Row::pack(Some(Datum::UInt64(row.hashed())))), row))
        };
        keyed
    } else {
        collection.map(|row| (None, row))
    };

    fn warn_on_dups<T>(v: &[T], sink_id: GlobalId, from_id: GlobalId) {
        if v.len() > 1 {
            warn!(
                sink_id =? sink_id,
                from_id =? from_id,
                "primary key error: expected at most one update per key and timestamp \
                This can happen when the configured sink key is not a primary key of \
                the sinked relation."
            )
        }
    }

    // Apply the envelope.
    // * "Debezium" consolidates the stream, sorts it by time, and produces DiffPairs from it.
    //   It then renders those as Avro.
    // * Upsert" does the same, except at the last step, it renders the diff pair in upsert format.
    //   (As part of doing so, it asserts that there are not multiple conflicting values at the same timestamp)
    let collection = match sink.envelope {
        SinkEnvelope::Debezium => {
            // Allow access to `arrange_named` because we cannot access Mz's wrapper from here.
            // TODO(#17413): Revisit with cluster unification.
            #[allow(clippy::disallowed_methods)]
            let combined = combine_at_timestamp(
                keyed.arrange_named::<ColValSpine<_, _, _, _>>("Arrange Debezium"),
            );

            // if there is no user-specified key, remove the synthetic
            // distribution key again
            let user_key_indices = sink_render.get_key_indices();
            let combined = if user_key_indices.is_some() {
                combined
            } else {
                combined.map(|(_key, value)| (None, value))
            };

            // This has to be an `Rc<RefCell<...>>` because the inner closure (passed to `Iterator::map`) references it, and it might outlive the outer closure.
            let row_buf = Rc::new(RefCell::new(Row::default()));
            let from_id = sink.from;
            let collection = combined.flat_map(move |(mut k, v)| {
                warn_on_dups(&v, sink_id, from_id);
                let max_idx = v.len() - 1;
                let row_buf = Rc::clone(&row_buf);
                v.into_iter().enumerate().map(move |(idx, dp)| {
                    let k = if idx == max_idx { k.take() } else { k.clone() };
                    let mut row_buf = row_buf.borrow_mut();
                    dbz_format(&mut row_buf.packer(), dp);
                    (k, Some(row_buf.clone()))
                })
            });
            collection
        }
        SinkEnvelope::Upsert => {
            // Allow access to `arrange_named` because we cannot access Mz's wrapper from here.
            // TODO(#17413): Revisit with cluster unification.
            #[allow(clippy::disallowed_methods)]
            let combined = combine_at_timestamp(
                keyed.arrange_named::<ColValSpine<_, _, _, _>>("Arrange Upsert"),
            );

            let from_id = sink.from;
            let collection = combined.flat_map(move |(mut k, v)| {
                warn_on_dups(&v, sink_id, from_id);
                let max_idx = v.len() - 1;
                v.into_iter().enumerate().map(move |(idx, dp)| {
                    let k = if idx == max_idx { k.take() } else { k.clone() };
                    (k, dp.after)
                })
            });
            collection
        }
    };

    collection
}

/// A type that can be rendered as a dataflow sink.
pub(crate) trait SinkRender<G>
where
    G: Scope<Timestamp = Timestamp>,
{
    /// TODO
    fn uses_keys(&self) -> bool;
    /// TODO
    fn get_key_indices(&self) -> Option<&[usize]>;
    /// TODO
    fn get_relation_key_indices(&self) -> Option<&[usize]>;
    /// TODO
    fn render_sink(
        &self,
        storage_state: &mut StorageState,
        sink: &StorageSinkDesc<MetadataFilled, Timestamp>,
        sink_id: GlobalId,
        sinked_collection: Collection<G, (Option<Row>, Option<Row>), Diff>,
        err_collection: Collection<G, DataflowError, Diff>,
    ) -> (Stream<G, HealthStatusMessage>, Vec<PressOnDropButton>);
}

fn get_sink_render_for<G>(connection: &StorageSinkConnection) -> Box<dyn SinkRender<G>>
where
    G: Scope<Timestamp = Timestamp>,
{
    match connection {
        StorageSinkConnection::Kafka(connection) => Box::new(connection.clone()),
    }
}