1use 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 _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 task_name: String,
235 data_topic: String,
237 progress_topic: String,
239 progress_key: ProgressKey,
241 sink_version: u64,
243 partition_count: Arc<AtomicU64>,
245 _partition_count_task: AbortOnDropHandle<()>,
247 producer: ThreadedProducer<TunnelingClientContext<MzClientContext>>,
249 statistics: SinkStatistics,
251 staged_messages: u64,
254 staged_bytes: u64,
257 socket_timeout: Duration,
259 transaction_timeout: Duration,
261}
262
263impl TransactionalProducer {
264 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 options.insert("enable.idempotence", "true".into());
289 options.insert(
291 "compression.type",
292 connection.compression_type.to_librdkafka_option().into(),
293 );
294 options.insert("queue.buffering.max.kbytes", "2147483647".into());
297 options.insert("queue.buffering.max.messages", "0".into());
300 options.insert("queue.buffering.max.ms", format!("{}", 10));
302 options.insert(
304 "transaction.timeout.ms",
305 format!("{}", timeout_config.transaction_timeout.as_millis()),
306 );
307 options.insert("transactional.id", transactional_id);
309 options.insert("client.id", client_id);
311 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 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 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 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 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 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 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 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 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 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
557async 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#[derive(Debug, Clone, Serialize, Deserialize)]
604struct KafkaMessage {
605 hash: u64,
607 key: Option<Vec<u8>>,
609 value: Option<Vec<u8>>,
611 headers: Vec<KafkaHeader>,
613}
614
615#[derive(Debug, Clone, Serialize, Deserialize)]
617struct KafkaHeader {
618 key: String,
620 value: Option<Vec<u8>>,
622}
623
624fn 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 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 let overcompacted =
683 *resume_upper != [Timestamp::minimum()] &&
685 !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 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 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 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 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 if !PartialOrder::less_equal(&resume_upper, &progress) {
750 continue;
751 }
752 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 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 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 break;
803 }
804 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
861async 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 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 "group.id" => group_id,
901 "client.id" => client_id,
903 "enable.auto.commit" => "false".into(),
904 "auto.offset.reset" => "earliest".into(),
905 "enable.partition.eof" => "true".into(),
908 };
909
910 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 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 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 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 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 metrics.consumed_progress_records.set(0);
1008
1009 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 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 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 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 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 Offset::Invalid => lo,
1129 _ => bail!(
1130 "Consumer::position returned offset of wrong type: {:?}",
1131 position
1132 ),
1133 };
1134 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 let mut last_progress: Option<ProgressRecord> = None;
1166 loop {
1167 let current_position = get_position()?;
1168
1169 if current_position >= hi {
1170 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 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 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 Ok(last_progress)
1222}
1223
1224#[derive(Debug, PartialEq, Serialize, Deserialize)]
1231pub struct LegacyProgressRecord {
1232 #[serde(default, deserialize_with = "deserialize_some")]
1235 pub timestamp: Option<Option<Timestamp>>,
1236}
1237
1238fn 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#[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 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
1299async 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
1328async 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
1356fn 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 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 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 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 *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 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
1566fn evaluate_partition_by(partition_by: &MirScalarExpr, row: &[Datum]) -> u64 {
1574 let temp_storage = RowArena::new();
1582 match partition_by.eval(row, &temp_storage) {
1583 Ok(Datum::UInt64(u)) => u,
1584 Ok(datum) => {
1585 soft_assert_or_log!(datum.is_null(), "unexpected partition_by result: {datum:?}");
1588 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}