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
// 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::rc::Rc;
use std::sync::Arc;
use std::time::{Duration, Instant};

use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::trace::implementations::ord_neu::{
    ColValBatcher, ColValBuilder, ColValSpine,
};
use differential_dataflow::{AsCollection, Collection, Hashable};
use mz_interchange::avro::DiffPair;
use mz_interchange::envelopes::combine_at_timestamp;
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, 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::internal_control::InternalStorageCommand;
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 command_tx = Rc::clone(&storage_state.internal_cmd_tx);

    let (ok_collection, err_collection, persist_tokens) = persist_source::persist_source(
        scope,
        sink.from,
        Arc::clone(&storage_state.persist_clients),
        &storage_state.txns_ctx,
        storage_state.storage_configuration.config_set(),
        sink.from_storage_metadata.clone(),
        Some(sink.as_of.clone()),
        snapshot_mode,
        timely::progress::Antichain::new(),
        None,
        None,
        async {},
        move |error| {
            Box::pin(async move {
                let error = format!("storage_sink: {error}");
                tracing::info!("{error}");
                let mut command_tx = command_tx.borrow_mut();
                command_tx.broadcast(InternalStorageCommand::SuspendAndRestart {
                    id: sink_id,
                    reason: error,
                });
            })
        },
    );
    tokens.extend(persist_tokens);

    let ok_collection =
        zip_into_diff_pairs(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)
}

/// Zip the input to a sink so that updates to the same key appear as
/// `DiffPair`s.
fn zip_into_diff_pairs<G>(
    sink_id: GlobalId,
    sink: &StorageSinkDesc<MetadataFilled, mz_repr::Timestamp>,
    sink_render: &dyn SinkRender<G>,
    collection: Collection<G, Row, Diff>,
) -> Collection<G, (Option<Row>, DiffPair<Row>), Diff>
where
    G: Scope<Timestamp = Timestamp>,
{
    // We need to consolidate the collection and group records by their key.
    // We'll first attempt to use the explicitly declared key when the sink was
    // created. If no such key exists, we'll use a key of the sink's underlying
    // relation, if one exists.
    //
    // If no such key exists, we'll generate a synthetic key based on the hash
    // of the row, just for purposes of distributing work among workers. In this
    // case the key offers no uniqueness guarantee.

    let user_key_indices = sink_render.get_key_indices();
    let relation_key_indices = sink_render.get_relation_key_indices();
    let key_indices = user_key_indices
        .or(relation_key_indices)
        .map(|k| k.to_vec());
    let key_is_synthetic = key_indices.is_none();

    let collection = match key_indices {
        None => collection.map(|row| (Some(Row::pack(Some(Datum::UInt64(row.hashed())))), row)),
        Some(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)
            })
        }
    };

    // Group messages by key at each timestamp.
    //
    // Allow access to `arrange_named` because we cannot access Mz's wrapper
    // from here. TODO(database-issues#5046): Revisit with cluster unification.
    #[allow(clippy::disallowed_methods)]
    let mut collection =
        combine_at_timestamp(collection.arrange_named::<ColValBatcher<_,_,_,_>, ColValBuilder<_,_,_,_>, ColValSpine<_, _, _, _>>("Arrange Sink"));

    // If there is no user-specified key, remove the synthetic key.
    //
    // We don't want the synthetic key to appear in the sink's actual output; we
    // just needed a value to use to distribute work.
    if user_key_indices.is_none() {
        collection = collection.map(|(_key, value)| (None, value))
    }

    collection.flat_map({
        let mut last_warning = Instant::now();
        let from_id = sink.from;
        move |(mut k, vs)| {
            // If the key is not synthetic, emit a warning to internal logs if
            // we discover a primary key violation.
            //
            // TODO: put the sink in a user-visible errored state instead of
            // only logging internally. See:
            // https://github.com/MaterializeInc/database-issues/issues/5099.
            if !key_is_synthetic && vs.len() > 1 {
                // We rate limit how often we emit this warning to avoid
                // flooding logs.
                let now = Instant::now();
                if now.duration_since(last_warning) >= Duration::from_secs(10) {
                    last_warning = now;
                    warn!(
                        ?sink_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"
                    )
                }
            }

            let max_idx = vs.len() - 1;
            vs.into_iter().enumerate().map(move |(idx, dp)| {
                let k = if idx == max_idx { k.take() } else { k.clone() };
                (k, dp)
            })
        }
    })
}

/// A type that can be rendered as a dataflow sink.
pub(crate) trait SinkRender<G>
where
    G: Scope<Timestamp = Timestamp>,
{
    /// Gets the indexes of the columns that form the key that the user
    /// specified when creating the sink, if any.
    fn get_key_indices(&self) -> Option<&[usize]>;

    /// Gets the indexes of the columns that form a key of the sink's underlying
    /// relation, if such a key exists.
    fn get_relation_key_indices(&self) -> Option<&[usize]>;

    /// Renders the sink's dataflow.
    fn render_sink(
        &self,
        storage_state: &mut StorageState,
        sink: &StorageSinkDesc<MetadataFilled, Timestamp>,
        sink_id: GlobalId,
        sinked_collection: Collection<G, (Option<Row>, DiffPair<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()),
    }
}