Skip to main content

mz_storage/sink/
kafka.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//! Code to render the sink dataflow of a [`KafkaSinkConnection`]. The dataflow consists
11//! of two operators in order to take advantage of all the available workers.
12//!
13//! ```text
14//!        ┏━━━━━━━━━━━━━━┓
15//!        ┃   persist    ┃
16//!        ┃    source    ┃
17//!        ┗━━━━━━┯━━━━━━━┛
18//!               │ row data, the input to this module
19//!               │
20//!        ┏━━━━━━v━━━━━━┓
21//!        ┃    row      ┃
22//!        ┃   encoder   ┃
23//!        ┗━━━━━━┯━━━━━━┛
24//!               │ encoded data
25//!               │
26//!        ┏━━━━━━v━━━━━━┓
27//!        ┃    kafka    ┃ (single worker)
28//!        ┃    sink     ┃
29//!        ┗━━┯━━━━━━━━┯━┛
30//!   records │        │ uppers
31//!      ╭────v──╮ ╭───v──────╮
32//!      │ data  │ │ progress │  <- records and uppers are produced
33//!      │ topic │ │  topic   │     transactionally to both topics
34//!      ╰───────╯ ╰──────────╯
35//! ```
36//!
37//! # Encoding
38//!
39//! One part of the dataflow deals with encoding the rows that we read from persist. There isn't
40//! anything surprizing here, it is *almost* just a `Collection::map` with the exception of an
41//! initialization step that makes sure the schemas are published to the Schema Registry. After
42//! that step the operator just encodes each batch it receives record by record.
43//!
44//! # Sinking
45//!
46//! The other part of the dataflow, and what this module mostly deals with, is interacting with the
47//! Kafka cluster in order to transactionally commit batches (sets of records associated with a
48//! frontier). All the processing happens in a single worker and so all previously encoded records
49//! go through an exchange in order to arrive at the chosen worker. We may be able to improve this
50//! in the future by committing disjoint partitions of the key space for independent workers but
51//! for now we do the simple thing.
52//!
53//! ## Retries
54//!
55//! All of the retry logic heavy lifting is offloaded to `librdkafka` since it already implements
56//! the required behavior[1]. In particular we only ever enqueue records to its send queue and
57//! eventually call `commit_transaction` which will ensure that all queued messages are
58//! successfully delivered before the transaction is reported as committed.
59//!
60//! The only error that is possible during sending is that the queue is full. We are purposefully
61//! NOT handling this error and simply configure `librdkafka` with a very large queue. The reason
62//! for this choice is that the only choice for hanlding such an error ourselves would be to queue
63//! it, and there isn't a good argument about two small queues being better than one big one. If we
64//! reach the queue limit we simply error out the entire sink dataflow and start over.
65//!
66//! # Error handling
67//!
68//! Both the encoding operator and the sinking operator can produce a transient error that is wired
69//! up with our health monitoring and will trigger a restart of the sink dataflow.
70//!
71//! [1]: https://github.com/confluentinc/librdkafka/blob/master/INTRODUCTION.md#message-reliability
72
73use std::cell::RefCell;
74use std::cmp::Ordering;
75use std::collections::BTreeMap;
76use std::future::Future;
77use std::rc::Rc;
78use std::sync::atomic::AtomicU64;
79use std::sync::{Arc, Weak};
80use std::time::Duration;
81
82use crate::healthcheck::{HealthStatusMessage, HealthStatusUpdate, StatusNamespace};
83use crate::metrics::sink::kafka::KafkaSinkMetrics;
84use crate::render::sinks::SinkRender;
85use crate::statistics::SinkStatistics;
86use crate::storage_state::StorageState;
87use anyhow::{Context, anyhow, bail};
88use differential_dataflow::{AsCollection, Hashable, VecCollection};
89use futures::StreamExt;
90use maplit::btreemap;
91use mz_expr::MirScalarExpr;
92use mz_interchange::avro::{AvroEncoder, DiffPair};
93use mz_interchange::encode::Encode;
94use mz_interchange::envelopes::dbz_format;
95use mz_interchange::json::JsonEncoder;
96use mz_interchange::text_binary::{BinaryEncoder, TextEncoder};
97use mz_kafka_util::admin::EnsureTopicConfig;
98use mz_kafka_util::client::{
99    DEFAULT_FETCH_METADATA_TIMEOUT, GetPartitionsError, MzClientContext, TimeoutConfig,
100    TunnelingClientContext,
101};
102use mz_ore::cast::CastFrom;
103use mz_ore::collections::CollectionExt;
104use mz_ore::error::ErrorExt;
105use mz_ore::future::InTask;
106use mz_ore::soft_assert_or_log;
107use mz_ore::task::{self, AbortOnDropHandle};
108use mz_persist_client::Diagnostics;
109use mz_persist_client::write::WriteHandle;
110use mz_persist_types::codec_impls::UnitSchema;
111use mz_repr::{Datum, DatumVec, Diff, GlobalId, Row, RowArena, Timestamp};
112use mz_storage_client::sink::progress_key::ProgressKey;
113use mz_storage_types::StorageDiff;
114use mz_storage_types::configuration::StorageConfiguration;
115use mz_storage_types::controller::CollectionMetadata;
116use mz_storage_types::dyncfgs::{
117    KAFKA_BUFFERED_EVENT_RESIZE_THRESHOLD_ELEMENTS, SINK_ENSURE_TOPIC_CONFIG, SINK_PROGRESS_SEARCH,
118};
119use mz_storage_types::errors::{ContextCreationError, ContextCreationErrorExt, DataflowError};
120use mz_storage_types::sinks::{
121    KafkaSinkConnection, KafkaSinkFormatType, SinkEnvelope, StorageSinkDesc,
122};
123use mz_storage_types::sources::SourceData;
124use mz_timely_util::antichain::AntichainExt;
125use mz_timely_util::builder_async::{
126    Event, OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton,
127};
128use rdkafka::consumer::{BaseConsumer, Consumer, ConsumerContext};
129use rdkafka::error::KafkaError;
130use rdkafka::message::{Header, OwnedHeaders, ToBytes};
131use rdkafka::producer::{BaseRecord, Producer, ThreadedProducer};
132use rdkafka::types::RDKafkaErrorCode;
133use rdkafka::{Message, Offset, Statistics, TopicPartitionList};
134use serde::{Deserialize, Deserializer, Serialize, Serializer};
135use timely::PartialOrder;
136use timely::container::CapacityContainerBuilder;
137use timely::dataflow::channels::pact::{Exchange, Pipeline};
138use timely::dataflow::operators::vec::{Map, ToStream};
139use timely::dataflow::operators::{CapabilitySet, Concatenate};
140use timely::dataflow::{Scope, StreamVec};
141use timely::progress::{Antichain, Timestamp as _};
142use tokio::sync::watch;
143use tokio::time::{self, MissedTickBehavior};
144use tracing::{debug, error, info, warn};
145
146impl<G: Scope<Timestamp = Timestamp>> SinkRender<G> for KafkaSinkConnection {
147    fn get_key_indices(&self) -> Option<&[usize]> {
148        self.key_desc_and_indices
149            .as_ref()
150            .map(|(_desc, indices)| indices.as_slice())
151    }
152
153    fn get_relation_key_indices(&self) -> Option<&[usize]> {
154        self.relation_key_indices.as_deref()
155    }
156
157    fn render_sink(
158        &self,
159        storage_state: &mut StorageState,
160        sink: &StorageSinkDesc<CollectionMetadata, Timestamp>,
161        sink_id: GlobalId,
162        input: VecCollection<G, (Option<Row>, DiffPair<Row>), Diff>,
163        // TODO(benesch): errors should stream out through the sink,
164        // if we figure out a protocol for that.
165        _err_collection: VecCollection<G, DataflowError, Diff>,
166    ) -> (StreamVec<G, HealthStatusMessage>, Vec<PressOnDropButton>) {
167        let mut scope = input.scope();
168
169        let write_handle = {
170            let persist = Arc::clone(&storage_state.persist_clients);
171            let shard_meta = sink.to_storage_metadata.clone();
172            async move {
173                let client = persist.open(shard_meta.persist_location).await?;
174                let handle = client
175                    .open_writer(
176                        shard_meta.data_shard,
177                        Arc::new(shard_meta.relation_desc),
178                        Arc::new(UnitSchema),
179                        Diagnostics::from_purpose("sink handle"),
180                    )
181                    .await?;
182                Ok(handle)
183            }
184        };
185
186        let write_frontier = Rc::new(RefCell::new(Antichain::from_elem(Timestamp::minimum())));
187        storage_state
188            .sink_write_frontiers
189            .insert(sink_id, Rc::clone(&write_frontier));
190
191        let (encoded, encode_status, encode_token) = encode_collection(
192            format!("kafka-{sink_id}-{}-encode", self.format.get_format_name()),
193            input,
194            sink.envelope,
195            self.clone(),
196            storage_state.storage_configuration.clone(),
197        );
198
199        let metrics = storage_state.metrics.get_kafka_sink_metrics(sink_id);
200        let statistics = storage_state
201            .aggregated_statistics
202            .get_sink(&sink_id)
203            .expect("statistics initialized")
204            .clone();
205
206        let (sink_status, sink_token) = sink_collection(
207            format!("kafka-{sink_id}-sink"),
208            encoded,
209            sink_id,
210            self.clone(),
211            storage_state.storage_configuration.clone(),
212            sink,
213            metrics,
214            statistics,
215            write_handle,
216            write_frontier,
217        );
218
219        let running_status = Some(HealthStatusMessage {
220            id: None,
221            update: HealthStatusUpdate::Running,
222            namespace: StatusNamespace::Kafka,
223        })
224        .to_stream(&mut scope);
225
226        let status = scope.concatenate([running_status, encode_status, sink_status]);
227
228        (status, vec![encode_token, sink_token])
229    }
230}
231
232struct TransactionalProducer {
233    /// The task name used for any blocking calls spawned onto the tokio threadpool.
234    task_name: String,
235    /// The topic where all the updates go.
236    data_topic: String,
237    /// The topic where all the upper frontiers go.
238    progress_topic: String,
239    /// The key each progress record is associated with.
240    progress_key: ProgressKey,
241    /// The version of this sink, used to fence out previous versions from writing.
242    sink_version: u64,
243    /// The number of partitions in the target topic.
244    partition_count: Arc<AtomicU64>,
245    /// A task to periodically refresh the partition count.
246    _partition_count_task: AbortOnDropHandle<()>,
247    /// The underlying Kafka producer.
248    producer: ThreadedProducer<TunnelingClientContext<MzClientContext>>,
249    /// A handle to the metrics associated with this sink.
250    statistics: SinkStatistics,
251    /// The number of messages staged for the currently open transactions. It is reset to zero
252    /// every time a transaction commits.
253    staged_messages: u64,
254    /// The total number bytes staged for the currently open transactions. It is reset to zero
255    /// every time a transaction commits.
256    staged_bytes: u64,
257    /// The timeout to use for network operations.
258    socket_timeout: Duration,
259    /// The timeout to use for committing transactions.
260    transaction_timeout: Duration,
261}
262
263impl TransactionalProducer {
264    /// Initializes a transcational producer for the sink identified by `sink_id`. After this call
265    /// returns it is guranteed that all previous `TransactionalProducer` instances for the same
266    /// sink have been fenced out (i.e `init_transations()` has been called successfully).
267    async fn new(
268        sink_id: GlobalId,
269        connection: &KafkaSinkConnection,
270        storage_configuration: &StorageConfiguration,
271        metrics: Arc<KafkaSinkMetrics>,
272        statistics: SinkStatistics,
273        sink_version: u64,
274    ) -> Result<(Self, Antichain<mz_repr::Timestamp>), ContextCreationError> {
275        let client_id = connection.client_id(
276            storage_configuration.config_set(),
277            &storage_configuration.connection_context,
278            sink_id,
279        );
280        let transactional_id =
281            connection.transactional_id(&storage_configuration.connection_context, sink_id);
282
283        let timeout_config = &storage_configuration.parameters.kafka_timeout_config;
284        let mut options = BTreeMap::new();
285        // Ensure that messages are sinked in order and without duplicates. Note that this only
286        // applies to a single instance of a producer - in the case of restarts, all bets are off
287        // and full exactly once support is required.
288        options.insert("enable.idempotence", "true".into());
289        // Use the compression type requested by the user.
290        options.insert(
291            "compression.type",
292            connection.compression_type.to_librdkafka_option().into(),
293        );
294        // Set the maximum buffer size limit. We don't want to impose anything lower than the max
295        // here as the operator has nothing better to do with the data than to buffer them.
296        options.insert("queue.buffering.max.kbytes", "2147483647".into());
297        // Disable the default buffer limit of 100k messages. We don't want to impose any limit
298        // here as the operator has nothing better to do with the data than to buffer them.
299        options.insert("queue.buffering.max.messages", "0".into());
300        // Make the Kafka producer wait at least 10 ms before sending out MessageSets
301        options.insert("queue.buffering.max.ms", format!("{}", 10));
302        // Time out transactions after 60 seconds
303        options.insert(
304            "transaction.timeout.ms",
305            format!("{}", timeout_config.transaction_timeout.as_millis()),
306        );
307        // Use the transactional ID requested by the user.
308        options.insert("transactional.id", transactional_id);
309        // Allow Kafka monitoring tools to identify this producer.
310        options.insert("client.id", client_id);
311        // We want to be notified regularly with statistics
312        options.insert("statistics.interval.ms", "1000".into());
313
314        let ctx = MzClientContext::default();
315
316        let stats_receiver = ctx.subscribe_statistics();
317        let task_name = format!("kafka_sink_metrics_collector:{sink_id}");
318        task::spawn(
319            || &task_name,
320            collect_statistics(stats_receiver, Arc::clone(&metrics)),
321        );
322
323        let producer: ThreadedProducer<_> = connection
324            .connection
325            .create_with_context(storage_configuration, ctx, &options, InTask::Yes)
326            .await?;
327
328        // The partition count is fixed up after we ensure the topic exists.
329        let partition_count = Arc::new(AtomicU64::new(0));
330        let update_partition_count = {
331            let partition_count = Arc::clone(&partition_count);
332            let metrics = Arc::clone(&metrics);
333            Arc::new(move |pc| {
334                partition_count.store(pc, std::sync::atomic::Ordering::SeqCst);
335                metrics.partition_count.set(pc);
336            })
337        };
338
339        // Start a task that will keep the partition count up to date in the
340        // background.
341        let partition_count_task = task::spawn(
342            || format!("kafka_sink_producer_fetch_metadata_loop:{sink_id}"),
343            fetch_partition_count_loop(
344                producer.clone(),
345                sink_id,
346                connection.topic.clone(),
347                connection.topic_metadata_refresh_interval,
348                Arc::clone(&update_partition_count),
349            ),
350        );
351
352        let task_name = format!("kafka_sink_producer:{sink_id}");
353        let progress_key = ProgressKey::new(sink_id);
354
355        let producer = Self {
356            task_name,
357            data_topic: connection.topic.clone(),
358            partition_count,
359            _partition_count_task: partition_count_task.abort_on_drop(),
360            progress_topic: connection
361                .progress_topic(&storage_configuration.connection_context)
362                .into_owned(),
363            progress_key,
364            sink_version,
365            producer,
366            statistics,
367            staged_messages: 0,
368            staged_bytes: 0,
369            socket_timeout: timeout_config.socket_timeout,
370            transaction_timeout: timeout_config.transaction_timeout,
371        };
372
373        let timeout = timeout_config.socket_timeout;
374        producer
375            .spawn_blocking(move |p| p.init_transactions(timeout))
376            .await?;
377
378        // We have just called init_transactions, which means that we have fenced out all previous
379        // transactional producers, making it safe to determine the resume upper.
380        let progress = determine_sink_progress(
381            sink_id,
382            connection,
383            storage_configuration,
384            Arc::clone(&metrics),
385        )
386        .await?;
387
388        let resume_upper = match progress {
389            Some(progress) => {
390                if sink_version < progress.version {
391                    return Err(ContextCreationError::Other(anyhow!(
392                        "Fenced off by newer version of the sink. ours={} theirs={}",
393                        sink_version,
394                        progress.version
395                    )));
396                }
397                progress.frontier
398            }
399            None => {
400                mz_storage_client::sink::ensure_kafka_topic(
401                    connection,
402                    storage_configuration,
403                    &connection.topic,
404                    &connection.topic_options,
405                    EnsureTopicConfig::Skip,
406                )
407                .await?;
408                Antichain::from_elem(Timestamp::minimum())
409            }
410        };
411
412        // At this point the topic must exist and so we can query for its
413        // partition count. Even though we have a background task to fetch the
414        // partition count, we do this synchronously to ensure we don't attempt
415        // to produce any messages with our initial partition count of 0.
416        let partition_count =
417            fetch_partition_count(&producer.producer, sink_id, &connection.topic).await?;
418        update_partition_count(partition_count);
419
420        Ok((producer, resume_upper))
421    }
422
423    /// Runs the blocking operation `f` on the producer in the tokio threadpool and checks for SSH
424    /// status in case of failure.
425    async fn spawn_blocking<F, R>(&self, f: F) -> Result<R, ContextCreationError>
426    where
427        F: FnOnce(
428                ThreadedProducer<TunnelingClientContext<MzClientContext>>,
429            ) -> Result<R, KafkaError>
430            + Send
431            + 'static,
432        R: Send + 'static,
433    {
434        let producer = self.producer.clone();
435        task::spawn_blocking(|| &self.task_name, move || f(producer))
436            .await
437            .check_ssh_status(self.producer.context())
438    }
439
440    async fn begin_transaction(&self) -> Result<(), ContextCreationError> {
441        self.spawn_blocking(|p| p.begin_transaction()).await
442    }
443
444    /// Synchronously puts the provided message to librdkafka's send queue. This method only
445    /// returns an error if the queue is full. Handling this error by buffering the message and
446    /// retrying is equivalent to adjusting the maximum number of queued items in rdkafka so it is
447    /// adviced that callers only handle this error in order to apply backpressure to the rest of
448    /// the system.
449    fn send(
450        &mut self,
451        message: &KafkaMessage,
452        time: Timestamp,
453        diff: Diff,
454    ) -> Result<(), KafkaError> {
455        assert_eq!(diff, Diff::ONE, "invalid sink update");
456
457        let mut headers = OwnedHeaders::new().insert(Header {
458            key: "materialize-timestamp",
459            value: Some(time.to_string().as_bytes()),
460        });
461        for header in &message.headers {
462            // Headers that start with `materialize-` are reserved for our
463            // internal use, so we silently drop any such user-specified
464            // headers. While this behavior is documented, it'd be a nicer UX to
465            // send a warning or error somewhere. Unfortunately sinks don't have
466            // anywhere user-visible to send errors. See database-issues#5148.
467            if header.key.starts_with("materialize-") {
468                continue;
469            }
470
471            headers = headers.insert(Header {
472                key: header.key.as_str(),
473                value: header.value.as_ref(),
474            });
475        }
476
477        let pc = self
478            .partition_count
479            .load(std::sync::atomic::Ordering::SeqCst);
480        let partition = Some(i32::try_from(message.hash % pc).unwrap());
481
482        let record = BaseRecord {
483            topic: &self.data_topic,
484            key: message.key.as_ref(),
485            payload: message.value.as_ref(),
486            headers: Some(headers),
487            partition,
488            timestamp: None,
489            delivery_opaque: (),
490        };
491        let key_size = message.key.as_ref().map(|k| k.len()).unwrap_or(0);
492        let value_size = message.value.as_ref().map(|k| k.len()).unwrap_or(0);
493        let headers_size = message
494            .headers
495            .iter()
496            .map(|h| h.key.len() + h.value.as_ref().map(|v| v.len()).unwrap_or(0))
497            .sum::<usize>();
498        let record_size = u64::cast_from(key_size + value_size + headers_size);
499        self.statistics.inc_messages_staged_by(1);
500        self.staged_messages += 1;
501        self.statistics.inc_bytes_staged_by(record_size);
502        self.staged_bytes += record_size;
503        self.producer.send(record).map_err(|(e, _)| e)
504    }
505
506    /// Commits all the staged updates of the currently open transaction plus a progress record
507    /// describing `upper` to the progress topic.
508    async fn commit_transaction(
509        &mut self,
510        upper: Antichain<Timestamp>,
511    ) -> Result<(), ContextCreationError> {
512        let progress = ProgressRecord {
513            frontier: upper,
514            version: self.sink_version,
515        };
516        let payload = serde_json::to_vec(&progress).expect("infallible");
517        let record = BaseRecord::to(&self.progress_topic)
518            .payload(&payload)
519            .key(&self.progress_key);
520        self.producer.send(record).map_err(|(e, _)| e)?;
521
522        fail::fail_point!("kafka_sink_commit_transaction");
523
524        let timeout = self.transaction_timeout;
525        match self
526            .spawn_blocking(move |p| p.commit_transaction(timeout))
527            .await
528        {
529            Ok(()) => {
530                self.statistics
531                    .inc_messages_committed_by(self.staged_messages);
532                self.statistics.inc_bytes_committed_by(self.staged_bytes);
533                self.staged_messages = 0;
534                self.staged_bytes = 0;
535                Ok(())
536            }
537            Err(ContextCreationError::KafkaError(KafkaError::Transaction(err))) => {
538                // Make one attempt at aborting the transaction before letting the error percolate
539                // up and the process exit. Aborting allows the consumers of the topic to skip over
540                // any messages we've written in the transaction, so it's polite to do... but if it
541                // fails, the transaction will be aborted either when fenced out by a future
542                // version of this producer or by the broker-side timeout.
543                if err.txn_requires_abort() {
544                    let timeout = self.socket_timeout;
545                    self.spawn_blocking(move |p| p.abort_transaction(timeout))
546                        .await?;
547                }
548                Err(ContextCreationError::KafkaError(KafkaError::Transaction(
549                    err,
550                )))
551            }
552            Err(err) => Err(err),
553        }
554    }
555}
556
557/// Listens for statistics updates from librdkafka and updates our Prometheus metrics.
558async fn collect_statistics(
559    mut receiver: watch::Receiver<Statistics>,
560    metrics: Arc<KafkaSinkMetrics>,
561) {
562    let mut outbuf_cnt: i64 = 0;
563    let mut outbuf_msg_cnt: i64 = 0;
564    let mut waitresp_cnt: i64 = 0;
565    let mut waitresp_msg_cnt: i64 = 0;
566    let mut txerrs: u64 = 0;
567    let mut txretries: u64 = 0;
568    let mut req_timeouts: u64 = 0;
569    let mut connects: i64 = 0;
570    let mut disconnects: i64 = 0;
571    while receiver.changed().await.is_ok() {
572        let stats = receiver.borrow();
573        for broker in stats.brokers.values() {
574            outbuf_cnt += broker.outbuf_cnt;
575            outbuf_msg_cnt += broker.outbuf_msg_cnt;
576            waitresp_cnt += broker.waitresp_cnt;
577            waitresp_msg_cnt += broker.waitresp_msg_cnt;
578            txerrs += broker.txerrs;
579            txretries += broker.txretries;
580            req_timeouts += broker.req_timeouts;
581            connects += broker.connects.unwrap_or(0);
582            disconnects += broker.disconnects.unwrap_or(0);
583        }
584        metrics.rdkafka_msg_cnt.set(stats.msg_cnt);
585        metrics.rdkafka_msg_size.set(stats.msg_size);
586        metrics.rdkafka_txmsgs.set(stats.txmsgs);
587        metrics.rdkafka_txmsg_bytes.set(stats.txmsg_bytes);
588        metrics.rdkafka_tx.set(stats.tx);
589        metrics.rdkafka_tx_bytes.set(stats.tx_bytes);
590        metrics.rdkafka_outbuf_cnt.set(outbuf_cnt);
591        metrics.rdkafka_outbuf_msg_cnt.set(outbuf_msg_cnt);
592        metrics.rdkafka_waitresp_cnt.set(waitresp_cnt);
593        metrics.rdkafka_waitresp_msg_cnt.set(waitresp_msg_cnt);
594        metrics.rdkafka_txerrs.set(txerrs);
595        metrics.rdkafka_txretries.set(txretries);
596        metrics.rdkafka_req_timeouts.set(req_timeouts);
597        metrics.rdkafka_connects.set(connects);
598        metrics.rdkafka_disconnects.set(disconnects);
599    }
600}
601
602/// A message to produce to Kafka.
603#[derive(Debug, Clone, Serialize, Deserialize)]
604struct KafkaMessage {
605    /// A hash of the key that can be used for partitioning.
606    hash: u64,
607    /// The message key.
608    key: Option<Vec<u8>>,
609    /// The message value.
610    value: Option<Vec<u8>>,
611    /// Message headers.
612    headers: Vec<KafkaHeader>,
613}
614
615/// A header to attach to a Kafka message.
616#[derive(Debug, Clone, Serialize, Deserialize)]
617struct KafkaHeader {
618    /// The header key.
619    key: String,
620    /// The header value.
621    value: Option<Vec<u8>>,
622}
623
624/// Sinks a collection of encoded rows to Kafka.
625///
626/// This operator exchanges all updates to a single worker by hashing on the given sink `id`.
627///
628/// Updates are sent in ascending timestamp order.
629fn sink_collection<G: Scope<Timestamp = Timestamp>>(
630    name: String,
631    input: VecCollection<G, KafkaMessage, Diff>,
632    sink_id: GlobalId,
633    connection: KafkaSinkConnection,
634    storage_configuration: StorageConfiguration,
635    sink: &StorageSinkDesc<CollectionMetadata, Timestamp>,
636    metrics: KafkaSinkMetrics,
637    statistics: SinkStatistics,
638    write_handle: impl Future<
639        Output = anyhow::Result<WriteHandle<SourceData, (), Timestamp, StorageDiff>>,
640    > + 'static,
641    write_frontier: Rc<RefCell<Antichain<Timestamp>>>,
642) -> (StreamVec<G, HealthStatusMessage>, PressOnDropButton) {
643    let scope = input.scope();
644    let mut builder = AsyncOperatorBuilder::new(name.clone(), input.inner.scope());
645
646    // We want exactly one worker to send all the data to the sink topic.
647    let hashed_id = sink_id.hashed();
648    let is_active_worker = usize::cast_from(hashed_id) % scope.peers() == scope.index();
649    let buffer_min_capacity =
650        KAFKA_BUFFERED_EVENT_RESIZE_THRESHOLD_ELEMENTS.handle(storage_configuration.config_set());
651
652    let mut input = builder.new_disconnected_input(input.inner, Exchange::new(move |_| hashed_id));
653
654    let as_of = sink.as_of.clone();
655    let sink_version = sink.version;
656    let (button, errors) = builder.build_fallible(move |_caps| {
657        Box::pin(async move {
658            if !is_active_worker {
659                write_frontier.borrow_mut().clear();
660                return Ok(());
661            }
662
663            fail::fail_point!("kafka_sink_creation_error", |_| Err(
664                ContextCreationError::Other(anyhow::anyhow!("synthetic error"))
665            ));
666
667            let mut write_handle = write_handle.await?;
668
669            let metrics = Arc::new(metrics);
670
671            let (mut producer, resume_upper) = TransactionalProducer::new(
672                sink_id,
673                &connection,
674                &storage_configuration,
675                Arc::clone(&metrics),
676                statistics,
677                sink_version,
678            )
679            .await?;
680
681            // The input has overcompacted if
682            let overcompacted =
683                // ..we have made some progress in the past
684                *resume_upper != [Timestamp::minimum()] &&
685                // ..but the since frontier is now beyond that
686                !PartialOrder::less_equal(&as_of, &resume_upper);
687            if overcompacted {
688                let err = format!(
689                    "{name}: input compacted past resume upper: as_of {}, resume_upper: {}",
690                    as_of.pretty(),
691                    resume_upper.pretty()
692                );
693                // This would normally be an assertion but because it can happen after a
694                // Materialize backup/restore we log an error so that it appears on Sentry but
695                // leaves the rest of the objects in the cluster unaffected.
696                error!("{err}");
697                return Err(anyhow!("{err}").into());
698            }
699
700            info!(
701                "{name}: as_of: {}, resume upper: {}",
702                as_of.pretty(),
703                resume_upper.pretty()
704            );
705
706            // The section below relies on TotalOrder for correctness so we'll work with timestamps
707            // directly to make sure this doesn't compile if someone attempts to make this operator
708            // generic over partial orders in the future.
709            let Some(mut upper) = resume_upper.clone().into_option() else {
710                write_frontier.borrow_mut().clear();
711                return Ok(());
712            };
713
714            let mut deferred_updates = vec![];
715            let mut extra_updates = vec![];
716            // We must wait until we have data to commit before starting a transaction because
717            // Kafka doesn't have a heartbeating mechanism to keep a transaction open indefinitely.
718            // This flag tracks whether we have started the transaction.
719            let mut transaction_begun = false;
720            while let Some(event) = input.next().await {
721                match event {
722                    Event::Data(_cap, batch) => {
723                        for (message, time, diff) in batch {
724                            // We want to publish updates in time order and we know that we have
725                            // already committed all times not beyond `upper`. Therefore, if this
726                            // update happens *exactly* at upper then it is the minimum pending
727                            // time and so emitting it now will not violate the timestamp publish
728                            // order. This optimization is load bearing because it is the mechanism
729                            // by which we incrementally stream the initial snapshot out to Kafka
730                            // instead of buffering it all in memory first. This argument doesn't
731                            // hold for partially ordered time because many different timestamps
732                            // can be *exactly* at upper but we can't know ahead of time which one
733                            // will be advanced in the next progress message.
734                            match upper.cmp(&time) {
735                                Ordering::Less => deferred_updates.push((message, time, diff)),
736                                Ordering::Equal => {
737                                    if !transaction_begun {
738                                        producer.begin_transaction().await?;
739                                        transaction_begun = true;
740                                    }
741                                    producer.send(&message, time, diff)?;
742                                }
743                                Ordering::Greater => continue,
744                            }
745                        }
746                    }
747                    Event::Progress(progress) => {
748                        // Ignore progress updates before our resumption frontier
749                        if !PartialOrder::less_equal(&resume_upper, &progress) {
750                            continue;
751                        }
752                        // Also ignore progress updates until we are past the as_of frontier. This
753                        // is to avoid the following pathological scenario:
754                        // 1. Sink gets instantiated with an as_of = {10}, resume_upper = {0}.
755                        //    `progress` initially jumps at {10}, then the snapshot appears at time
756                        //    10.
757                        // 2. `progress` would normally advance to say {11} and we would commit the
758                        //    snapshot but clusterd crashes instead.
759                        // 3. A new cluster restarts the sink with an earlier as_of, say {5}. This
760                        //    is valid, the earlier as_of has strictly more information. The
761                        //    snapshot now appears at time 5.
762                        //
763                        // If we were to commit an empty transaction in step 1 and advanced the
764                        // resume_upper to {10} then in step 3 we would ignore the snapshot that
765                        // now appears at 5 completely. So it is important to only start committing
766                        // transactions after we're strictly beyond the as_of.
767                        // TODO(petrosagg): is this logic an indication of us holding something
768                        // wrong elsewhere? Investigate.
769                        // Note: !PartialOrder::less_than(as_of, progress) would not be equivalent
770                        // nor correct for partially ordered times.
771                        if !as_of.iter().all(|t| !progress.less_equal(t)) {
772                            continue;
773                        }
774                        if !transaction_begun {
775                            producer.begin_transaction().await?;
776                        }
777
778                        // N.B. Shrinking the Vec here is important because when starting the Sink
779                        // we might buffer a ton of updates into these collections, e.g. if someone
780                        // deleted the progress topic and the resume upper is 0, and we don't want
781                        // to keep around a massively oversized VEc.
782                        deferred_updates.shrink_to(buffer_min_capacity.get());
783                        extra_updates.extend(
784                            deferred_updates
785                                .extract_if(.., |(_, time, _)| !progress.less_equal(time)),
786                        );
787                        extra_updates.sort_unstable_by(|a, b| a.1.cmp(&b.1));
788
789                        // N.B. See the comment above.
790                        extra_updates.shrink_to(buffer_min_capacity.get());
791                        for (message, time, diff) in extra_updates.drain(..) {
792                            producer.send(&message, time, diff)?;
793                        }
794
795                        debug!("{name}: committing transaction for {}", progress.pretty());
796                        producer.commit_transaction(progress.clone()).await?;
797                        transaction_begun = false;
798                        let mut expect_upper = write_handle.shared_upper();
799                        loop {
800                            if PartialOrder::less_equal(&progress, &expect_upper) {
801                                // The frontier has already been advanced as far as necessary.
802                                break;
803                            }
804                            // TODO(sinks): include the high water mark in the output topic for
805                            // the messages we've published, if and when we allow reads to the sink
806                            // directly, to allow monitoring the progress of the sink in terms of
807                            // the output system.
808                            const EMPTY: &[((SourceData, ()), Timestamp, StorageDiff)] = &[];
809                            match write_handle
810                                .compare_and_append(EMPTY, expect_upper, progress.clone())
811                                .await
812                                .expect("valid usage")
813                            {
814                                Ok(()) => break,
815                                Err(mismatch) => {
816                                    expect_upper = mismatch.current;
817                                }
818                            }
819                        }
820                        write_frontier.borrow_mut().clone_from(&progress);
821                        match progress.into_option() {
822                            Some(new_upper) => upper = new_upper,
823                            None => break,
824                        }
825                    }
826                }
827            }
828            Ok(())
829        })
830    });
831
832    let statuses = errors.map(|error: Rc<ContextCreationError>| {
833        let hint = match *error {
834            ContextCreationError::KafkaError(KafkaError::Transaction(ref e)) => {
835                if e.is_retriable() && e.code() == RDKafkaErrorCode::OperationTimedOut {
836                    let hint = "If you're running a single Kafka broker, ensure that the configs \
837                        transaction.state.log.replication.factor, transaction.state.log.min.isr, \
838                        and offsets.topic.replication.factor are set to 1 on the broker";
839                    Some(hint.to_owned())
840                } else {
841                    None
842                }
843            }
844            _ => None,
845        };
846
847        HealthStatusMessage {
848            id: None,
849            update: HealthStatusUpdate::halting(format!("{}", error.display_with_causes()), hint),
850            namespace: if matches!(*error, ContextCreationError::Ssh(_)) {
851                StatusNamespace::Ssh
852            } else {
853                StatusNamespace::Kafka
854            },
855        }
856    });
857
858    (statuses, button.press_on_drop())
859}
860
861/// Determines the latest progress record from the specified topic for the given
862/// progress key.
863///
864/// IMPORTANT: to achieve exactly once guarantees, the producer that will resume
865/// production at the returned timestamp *must* have called `init_transactions`
866/// prior to calling this method.
867async fn determine_sink_progress(
868    sink_id: GlobalId,
869    connection: &KafkaSinkConnection,
870    storage_configuration: &StorageConfiguration,
871    metrics: Arc<KafkaSinkMetrics>,
872) -> Result<Option<ProgressRecord>, ContextCreationError> {
873    // ****************************** WARNING ******************************
874    // Be VERY careful when editing the code in this function. It is very easy
875    // to accidentally introduce a correctness or liveness bug when refactoring
876    // this code.
877    // ****************************** WARNING ******************************
878
879    let TimeoutConfig {
880        fetch_metadata_timeout,
881        progress_record_fetch_timeout,
882        ..
883    } = storage_configuration.parameters.kafka_timeout_config;
884
885    let client_id = connection.client_id(
886        storage_configuration.config_set(),
887        &storage_configuration.connection_context,
888        sink_id,
889    );
890    let group_id = connection.progress_group_id(&storage_configuration.connection_context, sink_id);
891    let progress_topic = connection
892        .progress_topic(&storage_configuration.connection_context)
893        .into_owned();
894    let progress_topic_options = &connection.connection.progress_topic_options;
895    let progress_key = ProgressKey::new(sink_id);
896
897    let common_options = btreemap! {
898        // Consumer group ID, which may have been overridden by the user. librdkafka requires this,
899        // even though we'd prefer to disable the consumer group protocol entirely.
900        "group.id" => group_id,
901        // Allow Kafka monitoring tools to identify this consumer.
902        "client.id" => client_id,
903        "enable.auto.commit" => "false".into(),
904        "auto.offset.reset" => "earliest".into(),
905        // The fetch loop below needs EOF notifications to reliably detect that we have reached the
906        // high watermark.
907        "enable.partition.eof" => "true".into(),
908    };
909
910    // Construct two cliens in read committed and read uncommitted isolations respectively. See
911    // comment below for an explanation on why we need it.
912    let progress_client_read_committed: BaseConsumer<_> = {
913        let mut opts = common_options.clone();
914        opts.insert("isolation.level", "read_committed".into());
915        let ctx = MzClientContext::default();
916        connection
917            .connection
918            .create_with_context(storage_configuration, ctx, &opts, InTask::Yes)
919            .await?
920    };
921
922    let progress_client_read_uncommitted: BaseConsumer<_> = {
923        let mut opts = common_options;
924        opts.insert("isolation.level", "read_uncommitted".into());
925        let ctx = MzClientContext::default();
926        connection
927            .connection
928            .create_with_context(storage_configuration, ctx, &opts, InTask::Yes)
929            .await?
930    };
931
932    let ctx = Arc::clone(progress_client_read_committed.client().context());
933
934    // Ensure the progress topic exists.
935    let ensure_topic_config =
936        match &*SINK_ENSURE_TOPIC_CONFIG.get(storage_configuration.config_set()) {
937            "skip" => EnsureTopicConfig::Skip,
938            "check" => EnsureTopicConfig::Check,
939            "alter" => EnsureTopicConfig::Alter,
940            _ => {
941                tracing::warn!(
942                    topic = progress_topic,
943                    "unexpected value for ensure-topic-config; skipping checks"
944                );
945                EnsureTopicConfig::Skip
946            }
947        };
948    mz_storage_client::sink::ensure_kafka_topic(
949        connection,
950        storage_configuration,
951        &progress_topic,
952        progress_topic_options,
953        ensure_topic_config,
954    )
955    .await
956    .add_context("error registering kafka progress topic for sink")?;
957
958    // We are about to spawn a blocking task that cannot be aborted by simply calling .abort() on
959    // its handle but we must be able to cancel it prompty so as to not leave long running
960    // operations around when interest to this task is lost. To accomplish this we create a shared
961    // token of which a weak reference is given to the task and a strong reference is held by the
962    // parent task. The task periodically checks if its weak reference is still valid before
963    // continuing its work.
964    let parent_token = Arc::new(());
965    let child_token = Arc::downgrade(&parent_token);
966    let task_name = format!("get_latest_ts:{sink_id}");
967    let sink_progress_search = SINK_PROGRESS_SEARCH.get(storage_configuration.config_set());
968    let result = task::spawn_blocking(|| task_name, move || {
969        let progress_topic = progress_topic.as_ref();
970        // Ensure the progress topic has exactly one partition. Kafka only
971        // guarantees ordering within a single partition, and we need a strict
972        // order on the progress messages we read and write.
973        let partitions = match mz_kafka_util::client::get_partitions(
974            progress_client_read_committed.client(),
975            progress_topic,
976            fetch_metadata_timeout,
977        ) {
978            Ok(partitions) => partitions,
979            Err(GetPartitionsError::TopicDoesNotExist) => {
980                // The progress topic doesn't exist, which indicates there is
981                // no committed timestamp.
982                return Ok(None);
983            }
984            e => e.with_context(|| {
985                format!(
986                    "Unable to fetch metadata about progress topic {}",
987                    progress_topic
988                )
989            })?,
990        };
991        if partitions.len() != 1 {
992            bail!(
993                    "Progress topic {} should contain a single partition, but instead contains {} partitions",
994                    progress_topic, partitions.len(),
995                );
996        }
997        let partition = partitions.into_element();
998
999        // We scan from the beginning and see if we can find a progress record. We have
1000        // to do it like this because Kafka Control Batches mess with offsets. We
1001        // therefore cannot simply take the last offset from the back and expect a
1002        // progress message there. With a transactional producer, the OffsetTail(1) will
1003        // not point to an progress message but a control message. With aborted
1004        // transactions, there might even be a lot of garbage at the end of the
1005        // topic or in between.
1006
1007        metrics.consumed_progress_records.set(0);
1008
1009        // First, determine the current high water mark for the progress topic.
1010        // This is the position our `progress_client` consumer *must* reach
1011        // before we can conclude that we've seen the latest progress record for
1012        // the specified `progress_key`. A safety argument:
1013        //
1014        //   * Our caller has initialized transactions before calling this
1015        //     method, which prevents the prior incarnation of this sink from
1016        //     committing any further progress records.
1017        //
1018        //   * We use `read_uncommitted` isolation to ensure that we fetch the
1019        //     true high water mark for the topic, even if there are pending
1020        //     transactions in the topic. If we used the `read_committed`
1021        //     isolation level, we'd instead get the "last stable offset" (LSO),
1022        //     which is the offset of the first message in an open transaction,
1023        //     which might not include the last progress message committed for
1024        //     this sink! (While the caller of this function has fenced out
1025        //     older producers for this sink, *other* sinks writing using the
1026        //     same progress topic might have long-running transactions that
1027        //     hold back the LSO.)
1028        //
1029        //   * If another sink spins up and fences out the producer for this
1030        //     incarnation of the sink, we may not see the latest progress
1031        //     record... but since the producer has been fenced out, it will be
1032        //     unable to act on our stale information.
1033        //
1034        let (lo, hi) = progress_client_read_uncommitted
1035            .fetch_watermarks(progress_topic, partition, fetch_metadata_timeout)
1036            .map_err(|e| {
1037                anyhow!(
1038                    "Failed to fetch metadata while reading from progress topic: {}",
1039                    e
1040                )
1041            })?;
1042
1043        // This topic might be long, but the desired offset will usually be right near the end.
1044        // Instead of always scanning through the entire topic, we scan through exponentially-growing
1045        // suffixes of it. (Because writes are ordered, the largest progress record in any suffix,
1046        // if present, is the global max.) If we find it in one of our suffixes, we've saved at least
1047        // an order of magnitude of work; if we don't, we've added at most a constant factor.
1048        let mut start_indices = vec![lo];
1049        if sink_progress_search {
1050            let mut lookback = hi.saturating_sub(lo) / 10;
1051            while lookback >= 20_000 {
1052                start_indices.push(hi - lookback);
1053                lookback /= 10;
1054            }
1055        }
1056        for lo in start_indices.into_iter().rev() {
1057            if let Some(found) = progress_search(
1058                &progress_client_read_committed,
1059                progress_record_fetch_timeout,
1060                progress_topic,
1061                partition,
1062                lo,
1063                hi,
1064                progress_key.clone(),
1065                Weak::clone(&child_token),
1066                Arc::clone(&metrics)
1067            )? {
1068                return Ok(Some(found));
1069            }
1070        }
1071        Ok(None)
1072    }).await.check_ssh_status(&ctx);
1073    // Express interest to the computation until after we've received its result
1074    drop(parent_token);
1075    result
1076}
1077
1078fn progress_search<C: ConsumerContext + 'static>(
1079    progress_client_read_committed: &BaseConsumer<C>,
1080    progress_record_fetch_timeout: Duration,
1081    progress_topic: &str,
1082    partition: i32,
1083    lo: i64,
1084    hi: i64,
1085    progress_key: ProgressKey,
1086    child_token: Weak<()>,
1087    metrics: Arc<KafkaSinkMetrics>,
1088) -> anyhow::Result<Option<ProgressRecord>> {
1089    // Seek to the beginning of the given range in the progress topic.
1090    let mut tps = TopicPartitionList::new();
1091    tps.add_partition(progress_topic, partition);
1092    tps.set_partition_offset(progress_topic, partition, Offset::Offset(lo))?;
1093    progress_client_read_committed
1094        .assign(&tps)
1095        .with_context(|| {
1096            format!(
1097                "Error seeking in progress topic {}:{}",
1098                progress_topic, partition
1099            )
1100        })?;
1101
1102    // Helper to get the progress consumer's current position.
1103    let get_position = || {
1104        if child_token.strong_count() == 0 {
1105            bail!("operation cancelled");
1106        }
1107        let position = progress_client_read_committed
1108            .position()?
1109            .find_partition(progress_topic, partition)
1110            .ok_or_else(|| {
1111                anyhow!(
1112                    "No position info found for progress topic {}",
1113                    progress_topic
1114                )
1115            })?
1116            .offset();
1117        let position = match position {
1118            Offset::Offset(position) => position,
1119            // An invalid offset indicates the consumer has not yet read a
1120            // message. Since we assigned the consumer to the beginning of
1121            // the topic, it's safe to return the low water mark here, which
1122            // indicates the position before the first possible message.
1123            //
1124            // Note that it's important to return the low water mark and not
1125            // the minimum possible offset (i.e., zero) in order to break
1126            // out of the loop if the topic is empty but the low water mark
1127            // is greater than zero.
1128            Offset::Invalid => lo,
1129            _ => bail!(
1130                "Consumer::position returned offset of wrong type: {:?}",
1131                position
1132            ),
1133        };
1134        // Record the outstanding number of progress records that remain to be processed
1135        let outstanding = u64::try_from(std::cmp::max(0, hi - position)).unwrap();
1136        metrics.outstanding_progress_records.set(outstanding);
1137        Ok(position)
1138    };
1139
1140    info!("fetching latest progress record for {progress_key}, lo/hi: {lo}/{hi}");
1141
1142    // Read messages until the consumer is positioned at or beyond the high
1143    // water mark.
1144    //
1145    // We use `read_committed` isolation to ensure we don't see progress
1146    // records for transactions that did not commit. This means we have to
1147    // wait for the LSO to progress to the high water mark `hi`, which means
1148    // waiting for any open transactions for other sinks using the same
1149    // progress topic to complete. We set a short transaction timeout (10s)
1150    // to ensure we never need to wait more than 10s.
1151    //
1152    // Note that the stall time on the progress topic is not a function of
1153    // transaction size. We've designed our transactions so that the
1154    // progress record is always written last, after all the data has been
1155    // written, and so the window of time in which the progress topic has an
1156    // open transaction is quite small. The only vulnerability is if another
1157    // sink using the same progress topic crashes in that small window
1158    // between writing the progress record and committing the transaction,
1159    // in which case we have to wait out the transaction timeout.
1160    //
1161    // Important invariant: we only exit this loop successfully (i.e., not
1162    // returning an error) if we have positive proof of a position at or
1163    // beyond the high water mark. To make this invariant easy to check, do
1164    // not use `break` in the body of the loop.
1165    let mut last_progress: Option<ProgressRecord> = None;
1166    loop {
1167        let current_position = get_position()?;
1168
1169        if current_position >= hi {
1170            // consumer is at or beyond the high water mark and has read enough messages
1171            break;
1172        }
1173
1174        let message = match progress_client_read_committed.poll(progress_record_fetch_timeout) {
1175            Some(Ok(message)) => message,
1176            Some(Err(KafkaError::PartitionEOF(_))) => {
1177                // No message, but the consumer's position may have advanced
1178                // past a transaction control message that positions us at
1179                // or beyond the high water mark. Go around the loop again
1180                // to check.
1181                continue;
1182            }
1183            Some(Err(e)) => bail!("failed to fetch progress message {e}"),
1184            None => {
1185                bail!(
1186                    "timed out while waiting to reach high water mark of non-empty \
1187                        topic {progress_topic}:{partition}, lo/hi: {lo}/{hi}, current position: {current_position}"
1188                );
1189            }
1190        };
1191
1192        if message.key() != Some(progress_key.to_bytes()) {
1193            // This is a progress message for a different sink.
1194            continue;
1195        }
1196
1197        metrics.consumed_progress_records.inc();
1198
1199        let Some(payload) = message.payload() else {
1200            continue;
1201        };
1202        let progress = parse_progress_record(payload)?;
1203
1204        match last_progress {
1205            Some(last_progress)
1206                if !PartialOrder::less_equal(&last_progress.frontier, &progress.frontier) =>
1207            {
1208                bail!(
1209                    "upper regressed in topic {progress_topic}:{partition} from {:?} to {:?}",
1210                    &last_progress.frontier,
1211                    &progress.frontier,
1212                );
1213            }
1214            _ => last_progress = Some(progress),
1215        }
1216    }
1217
1218    // If we get here, we are assured that we've read all messages up to
1219    // the high water mark, and therefore `last_timestamp` contains the
1220    // most recent timestamp for the sink under consideration.
1221    Ok(last_progress)
1222}
1223
1224/// This is the legacy struct that used to be emitted as part of a transactional produce and
1225/// contains the largest timestamp within the batch committed. Since it is just a timestamp it
1226/// cannot encode the fact that a sink has finished and deviates from upper frontier semantics.
1227/// Materialize no longer produces this record but it's possible that we encounter this in topics
1228/// written by older versions. In those cases we convert it into upper semantics by stepping the
1229/// timestamp forward.
1230#[derive(Debug, PartialEq, Serialize, Deserialize)]
1231pub struct LegacyProgressRecord {
1232    // Double Option to tell apart an omitted field from one set to null explicitly
1233    // https://github.com/serde-rs/serde/issues/984
1234    #[serde(default, deserialize_with = "deserialize_some")]
1235    pub timestamp: Option<Option<Timestamp>>,
1236}
1237
1238// Any value that is present is considered Some value, including null.
1239fn deserialize_some<'de, T, D>(deserializer: D) -> Result<Option<T>, D::Error>
1240where
1241    T: Deserialize<'de>,
1242    D: Deserializer<'de>,
1243{
1244    Deserialize::deserialize(deserializer).map(Some)
1245}
1246
1247/// This struct is emitted as part of a transactional produce, and contains the upper frontier of
1248/// the batch committed. It is used to recover the frontier a sink needs to resume at.
1249#[derive(Debug, PartialEq, Serialize, Deserialize)]
1250pub struct ProgressRecord {
1251    #[serde(
1252        deserialize_with = "deserialize_frontier",
1253        serialize_with = "serialize_frontier"
1254    )]
1255    pub frontier: Antichain<Timestamp>,
1256    #[serde(default)]
1257    pub version: u64,
1258}
1259fn serialize_frontier<S>(frontier: &Antichain<Timestamp>, serializer: S) -> Result<S::Ok, S::Error>
1260where
1261    S: Serializer,
1262{
1263    Serialize::serialize(frontier.elements(), serializer)
1264}
1265
1266fn deserialize_frontier<'de, D>(deserializer: D) -> Result<Antichain<Timestamp>, D::Error>
1267where
1268    D: Deserializer<'de>,
1269{
1270    let times: Vec<Timestamp> = Deserialize::deserialize(deserializer)?;
1271    Ok(Antichain::from(times))
1272}
1273
1274fn parse_progress_record(payload: &[u8]) -> Result<ProgressRecord, anyhow::Error> {
1275    Ok(match serde_json::from_slice::<ProgressRecord>(payload) {
1276        Ok(progress) => progress,
1277        // If we fail to deserialize we might be reading a legacy progress record
1278        Err(_) => match serde_json::from_slice::<LegacyProgressRecord>(payload) {
1279            Ok(LegacyProgressRecord {
1280                timestamp: Some(Some(time)),
1281            }) => ProgressRecord {
1282                frontier: Antichain::from_elem(time.step_forward()),
1283                version: 0,
1284            },
1285            Ok(LegacyProgressRecord {
1286                timestamp: Some(None),
1287            }) => ProgressRecord {
1288                frontier: Antichain::new(),
1289                version: 0,
1290            },
1291            _ => match std::str::from_utf8(payload) {
1292                Ok(payload) => bail!("invalid progress record: {payload}"),
1293                Err(_) => bail!("invalid progress record bytes: {payload:?}"),
1294            },
1295        },
1296    })
1297}
1298
1299/// Fetches the partition count for the identified topic.
1300async fn fetch_partition_count(
1301    producer: &ThreadedProducer<TunnelingClientContext<MzClientContext>>,
1302    sink_id: GlobalId,
1303    topic_name: &str,
1304) -> Result<u64, anyhow::Error> {
1305    let meta = task::spawn_blocking(|| format!("kafka_sink_fetch_partition_count:{sink_id}"), {
1306        let producer = producer.clone();
1307        move || {
1308            producer
1309                .client()
1310                .fetch_metadata(None, DEFAULT_FETCH_METADATA_TIMEOUT)
1311        }
1312    })
1313    .await
1314    .check_ssh_status(producer.context())?;
1315
1316    match meta.topics().iter().find(|t| t.name() == topic_name) {
1317        Some(topic) => {
1318            let partition_count = u64::cast_from(topic.partitions().len());
1319            if partition_count == 0 {
1320                bail!("topic {topic_name} has an impossible partition count of zero");
1321            }
1322            Ok(partition_count)
1323        }
1324        None => bail!("topic {topic_name} does not exist"),
1325    }
1326}
1327
1328/// Fetches the partition count for the identified topic at the specified
1329/// interval.
1330///
1331/// When an updated partition count is discovered, invokes
1332/// `update_partition_count` with the new partition count.
1333async fn fetch_partition_count_loop<F>(
1334    producer: ThreadedProducer<TunnelingClientContext<MzClientContext>>,
1335    sink_id: GlobalId,
1336    topic_name: String,
1337    interval: Duration,
1338    update_partition_count: Arc<F>,
1339) where
1340    F: Fn(u64),
1341{
1342    let mut interval = time::interval(interval);
1343    interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
1344    loop {
1345        interval.tick().await;
1346        match fetch_partition_count(&producer, sink_id, &topic_name).await {
1347            Ok(pc) => update_partition_count(pc),
1348            Err(e) => {
1349                warn!(%sink_id, "failed updating partition count: {e}");
1350                continue;
1351            }
1352        };
1353    }
1354}
1355
1356/// Encodes a stream of `(Option<Row>, Option<Row>)` updates using the specified encoder.
1357///
1358/// Input [`Row`] updates must me compatible with the given implementor of [`Encode`].
1359fn encode_collection<G: Scope>(
1360    name: String,
1361    input: VecCollection<G, (Option<Row>, DiffPair<Row>), Diff>,
1362    envelope: SinkEnvelope,
1363    connection: KafkaSinkConnection,
1364    storage_configuration: StorageConfiguration,
1365) -> (
1366    VecCollection<G, KafkaMessage, Diff>,
1367    StreamVec<G, HealthStatusMessage>,
1368    PressOnDropButton,
1369) {
1370    let mut builder = AsyncOperatorBuilder::new(name, input.inner.scope());
1371
1372    let (output, stream) = builder.new_output::<CapacityContainerBuilder<Vec<_>>>();
1373    let mut input = builder.new_input_for(input.inner, Pipeline, &output);
1374
1375    let (button, errors) = builder.build_fallible(move |caps| {
1376        Box::pin(async move {
1377            let [capset]: &mut [_; 1] = caps.try_into().unwrap();
1378            let key_desc = connection
1379                .key_desc_and_indices
1380                .as_ref()
1381                .map(|(desc, _indices)| desc.clone());
1382            let value_desc = connection.value_desc;
1383
1384            let key_encoder: Option<Box<dyn Encode>> =
1385                match (key_desc, connection.format.key_format) {
1386                    (Some(desc), Some(KafkaSinkFormatType::Bytes)) => {
1387                        Some(Box::new(BinaryEncoder::new(desc, false)))
1388                    }
1389                    (Some(desc), Some(KafkaSinkFormatType::Text)) => {
1390                        Some(Box::new(TextEncoder::new(desc, false)))
1391                    }
1392                    (Some(desc), Some(KafkaSinkFormatType::Json)) => {
1393                        Some(Box::new(JsonEncoder::new(desc, false)))
1394                    }
1395                    (Some(desc), Some(KafkaSinkFormatType::Avro {
1396                        schema,
1397                        compatibility_level,
1398                        csr_connection,
1399                    })) => {
1400                        // Ensure that schemas are registered with the schema registry.
1401                        //
1402                        // Note that where this lies in the rendering cycle means that we will publish the
1403                        // schemas each time the sink is rendered.
1404                        let ccsr = csr_connection
1405                            .connect(&storage_configuration, InTask::Yes)
1406                            .await?;
1407
1408                        let schema_id = mz_storage_client::sink::publish_kafka_schema(
1409                            ccsr,
1410                            format!("{}-key", connection.topic),
1411                            schema.clone(),
1412                            mz_ccsr::SchemaType::Avro,
1413                            compatibility_level,
1414                        )
1415                        .await
1416                        .context("error publishing kafka schemas for sink")?;
1417
1418                        Some(Box::new(AvroEncoder::new(desc, false, &schema, schema_id)))
1419                    }
1420                    (None, None) => None,
1421                    (desc, format) => {
1422                        return Err(anyhow!(
1423                            "key_desc and key_format must be both set or both unset, but key_desc: {:?}, key_format: {:?}",
1424                            desc,
1425                            format
1426                        ))
1427                    }
1428                };
1429
1430            // whether to apply the debezium envelope to the value encoding
1431            let debezium = matches!(envelope, SinkEnvelope::Debezium);
1432
1433            let value_encoder: Box<dyn Encode> = match connection.format.value_format {
1434                KafkaSinkFormatType::Bytes => Box::new(BinaryEncoder::new(value_desc, debezium)),
1435                KafkaSinkFormatType::Text => Box::new(TextEncoder::new(value_desc, debezium)),
1436                KafkaSinkFormatType::Json => Box::new(JsonEncoder::new(value_desc, debezium)),
1437                KafkaSinkFormatType::Avro {
1438                    schema,
1439                    compatibility_level,
1440                    csr_connection,
1441                } => {
1442                    // Ensure that schemas are registered with the schema registry.
1443                    //
1444                    // Note that where this lies in the rendering cycle means that we will publish the
1445                    // schemas each time the sink is rendered.
1446                    let ccsr = csr_connection
1447                        .connect(&storage_configuration, InTask::Yes)
1448                        .await?;
1449
1450                    let schema_id = mz_storage_client::sink::publish_kafka_schema(
1451                        ccsr,
1452                        format!("{}-value", connection.topic),
1453                        schema.clone(),
1454                        mz_ccsr::SchemaType::Avro,
1455                        compatibility_level,
1456                    )
1457                    .await
1458                    .context("error publishing kafka schemas for sink")?;
1459
1460                    Box::new(AvroEncoder::new(value_desc, debezium, &schema, schema_id))
1461                }
1462            };
1463
1464            // !IMPORTANT!
1465            // Correctness of this operator relies on no fallible operations happening after this
1466            // point. This is a temporary workaround of build_fallible's bad interaction of owned
1467            // capabilities and errors.
1468            // TODO(petrosagg): Make the fallible async operator safe
1469            *capset = CapabilitySet::new();
1470
1471            let mut row_buf = Row::default();
1472            let mut datums = DatumVec::new();
1473
1474            while let Some(event) = input.next().await {
1475                if let Event::Data(cap, rows) = event {
1476                    for ((key, value), time, diff) in rows {
1477                        let mut hash = None;
1478                        let mut headers = vec![];
1479                        if connection.headers_index.is_some() || connection.partition_by.is_some() {
1480                            // Header values and partition by values are derived from the row that
1481                            // produces an event. But it is ambiguous whether to use the `before` or
1482                            // `after` from the event. The rule applied here is simple: use `after`
1483                            // if it exists (insertions and updates), otherwise fall back to `before`
1484                            // (deletions).
1485                            //
1486                            // It is up to the SQL planner to ensure this produces sensible results.
1487                            // (When using the upsert envelope and both `before` and `after` are
1488                            // present, it's always unambiguous to use `after` because that's all
1489                            // that will be present in the Kafka message; when using the Debezium
1490                            // envelope, it's okay to refer to columns in the key because those
1491                            // are guaranteed to be the same in both `before` and `after`.)
1492                            let row = value
1493                                .after
1494                                .as_ref()
1495                                .or(value.before.as_ref())
1496                                .expect("one of before or after must be set");
1497                            let row = datums.borrow_with(row);
1498
1499                            if let Some(i) = connection.headers_index {
1500                                headers = encode_headers(row[i]);
1501                            }
1502
1503                            if let Some(partition_by) = &connection.partition_by {
1504                                hash = Some(evaluate_partition_by(partition_by, &row));
1505                            }
1506                        }
1507                        let (key, hash) = match key {
1508                            Some(key) => {
1509                                let key_encoder = key_encoder.as_ref().expect("key present");
1510                                let key = key_encoder.encode_unchecked(key);
1511                                let hash = hash.unwrap_or_else(|| key_encoder.hash(&key));
1512                                (Some(key), hash)
1513                            }
1514                            None => (None, hash.unwrap_or(0))
1515                        };
1516                        let value = match envelope {
1517                            SinkEnvelope::Upsert => value.after,
1518                            SinkEnvelope::Debezium => {
1519                                dbz_format(&mut row_buf.packer(), value);
1520                                Some(row_buf.clone())
1521                            }
1522                        };
1523                        let value = value.map(|value| value_encoder.encode_unchecked(value));
1524                        let message = KafkaMessage {
1525                            hash,
1526                            key,
1527                            value,
1528                            headers,
1529                        };
1530                        output.give(&cap, (message, time, diff));
1531                    }
1532                }
1533            }
1534            Ok::<(), anyhow::Error>(())
1535        })
1536    });
1537
1538    let statuses = errors.map(|error| HealthStatusMessage {
1539        id: None,
1540        update: HealthStatusUpdate::halting(format!("{}", error.display_with_causes()), None),
1541        namespace: StatusNamespace::Kafka,
1542    });
1543
1544    (stream.as_collection(), statuses, button.press_on_drop())
1545}
1546
1547fn encode_headers(datum: Datum) -> Vec<KafkaHeader> {
1548    let mut out = vec![];
1549    if datum.is_null() {
1550        return out;
1551    }
1552    for (key, value) in datum.unwrap_map().iter() {
1553        out.push(KafkaHeader {
1554            key: key.into(),
1555            value: match value {
1556                Datum::Null => None,
1557                Datum::String(s) => Some(s.as_bytes().to_vec()),
1558                Datum::Bytes(b) => Some(b.to_vec()),
1559                _ => panic!("encode_headers called with unexpected header value {value:?}"),
1560            },
1561        })
1562    }
1563    out
1564}
1565
1566/// Evaluates a partition by expression on the given row, returning the hash
1567/// value to use for partition assignment.
1568///
1569/// The provided expression must have type `Int32`, `Int64`, `UInt32`, or
1570/// `UInt64`. If the expression produces an error when evaluated, or if the
1571/// expression is of a signed integer type and produces a negative value, this
1572/// function returns 0.
1573fn evaluate_partition_by(partition_by: &MirScalarExpr, row: &[Datum]) -> u64 {
1574    // NOTE(benesch): The way this function converts errors and invalid values
1575    // to 0 is somewhat surpising. Ideally, we would put the sink in a
1576    // permanently errored state if the partition by expression produces an
1577    // error or invalid value. But we don't presently have a way for sinks to
1578    // report errors (see materialize#17688), so the current behavior was determined to be
1579    // the best available option. The behavior is clearly documented in the
1580    // user-facing `CREATE SINK` docs.
1581    let temp_storage = RowArena::new();
1582    match partition_by.eval(row, &temp_storage) {
1583        Ok(Datum::UInt64(u)) => u,
1584        Ok(datum) => {
1585            // If we are here the only valid type that we should be seeing is
1586            // null. Anything else is a bug in the planner.
1587            soft_assert_or_log!(datum.is_null(), "unexpected partition_by result: {datum:?}");
1588            // We treat nulls the same as we treat errors: map them to partition 0.
1589            0
1590        }
1591        Err(_) => 0,
1592    }
1593}
1594
1595#[cfg(test)]
1596mod test {
1597    use mz_ore::assert_err;
1598
1599    use super::*;
1600
1601    #[mz_ore::test]
1602    fn progress_record_migration() {
1603        assert_err!(parse_progress_record(b"{}"));
1604
1605        assert_eq!(
1606            parse_progress_record(b"{\"timestamp\":1}").unwrap(),
1607            ProgressRecord {
1608                frontier: Antichain::from_elem(2.into()),
1609                version: 0,
1610            }
1611        );
1612
1613        assert_eq!(
1614            parse_progress_record(b"{\"timestamp\":null}").unwrap(),
1615            ProgressRecord {
1616                frontier: Antichain::new(),
1617                version: 0,
1618            }
1619        );
1620
1621        assert_eq!(
1622            parse_progress_record(b"{\"frontier\":[1]}").unwrap(),
1623            ProgressRecord {
1624                frontier: Antichain::from_elem(1.into()),
1625                version: 0,
1626            }
1627        );
1628
1629        assert_eq!(
1630            parse_progress_record(b"{\"frontier\":[]}").unwrap(),
1631            ProgressRecord {
1632                frontier: Antichain::new(),
1633                version: 0,
1634            }
1635        );
1636
1637        assert_eq!(
1638            parse_progress_record(b"{\"frontier\":[], \"version\": 42}").unwrap(),
1639            ProgressRecord {
1640                frontier: Antichain::new(),
1641                version: 42,
1642            }
1643        );
1644
1645        assert_err!(parse_progress_record(b"{\"frontier\":null}"));
1646    }
1647}