1
  2
  3
  4
  5
  6
  7
  8
  9
 10
 11
 12
 13
 14
 15
 16
 17
 18
 19
 20
 21
 22
 23
 24
 25
 26
 27
 28
 29
 30
 31
 32
 33
 34
 35
 36
 37
 38
 39
 40
 41
 42
 43
 44
 45
 46
 47
 48
 49
 50
 51
 52
 53
 54
 55
 56
 57
 58
 59
 60
 61
 62
 63
 64
 65
 66
 67
 68
 69
 70
 71
 72
 73
 74
 75
 76
 77
 78
 79
 80
 81
 82
 83
 84
 85
 86
 87
 88
 89
 90
 91
 92
 93
 94
 95
 96
 97
 98
 99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

use std::collections::{BTreeMap, HashMap, VecDeque};
use std::sync::{Arc, Mutex};
use std::time::Duration;

use rdkafka::consumer::base_consumer::PartitionQueue;
use rdkafka::consumer::{BaseConsumer, Consumer, ConsumerContext};
use rdkafka::error::KafkaError;
use rdkafka::message::BorrowedMessage;
use rdkafka::topic_partition_list::Offset;
use rdkafka::{ClientConfig, ClientContext, Message, TopicPartitionList};
use timely::scheduling::activate::SyncActivator;

use dataflow_types::sources::{
    encoding::SourceDataEncoding, ExternalSourceConnector, KafkaOffset, KafkaSourceConnector,
    MzOffset,
};
use expr::{PartitionId, SourceInstanceId};
use kafka_util::{client::MzClientContext, KafkaAddrs};
use repr::adt::jsonb::Jsonb;
use tracing::{debug, error, info, warn};
use uuid::Uuid;

use crate::logging::materialized::{Logger, MaterializedEvent};
use crate::source::{NextMessage, SourceMessage, SourceReader};

use super::metrics::SourceBaseMetrics;

/// Contains all information necessary to ingest data from Kafka
pub struct KafkaSourceReader {
    /// Name of the topic on which this source is backed on
    topic_name: String,
    /// Name of the source (will have format kafka-source-id)
    source_name: String,
    /// Source instance ID
    id: SourceInstanceId,
    /// Kafka consumer for this source
    consumer: Arc<BaseConsumer<GlueConsumerContext>>,
    /// List of consumers. A consumer should be assigned per partition to guarantee fairness
    partition_consumers: VecDeque<PartitionConsumer>,
    /// Worker ID
    worker_id: usize,
    /// Total count of workers
    worker_count: usize,
    /// Map from partition -> most recently read offset
    last_offsets: HashMap<i32, i64>,
    /// Map from partition -> offset to start reading at
    start_offsets: HashMap<i32, i64>,
    /// Timely worker logger for source events
    logger: Option<Logger>,
    /// Channel to receive Kafka statistics JSON blobs from the stats callback.
    stats_rx: crossbeam_channel::Receiver<Jsonb>,
    // The last statistics JSON blob received.
    last_stats: Option<Jsonb>,
    /// The last partition we received
    partition_info: Arc<Mutex<Option<Vec<i32>>>>,
}

impl SourceReader for KafkaSourceReader {
    type Key = Option<Vec<u8>>;
    type Value = Option<Vec<u8>>;

    /// Create a new instance of a Kafka reader.
    fn new(
        source_name: String,
        source_id: SourceInstanceId,
        worker_id: usize,
        worker_count: usize,
        consumer_activator: SyncActivator,
        connector: ExternalSourceConnector,
        restored_offsets: Vec<(PartitionId, Option<MzOffset>)>,
        _: SourceDataEncoding,
        logger: Option<Logger>,
        _: SourceBaseMetrics,
    ) -> Result<(KafkaSourceReader, Option<PartitionId>), anyhow::Error> {
        match connector {
            ExternalSourceConnector::Kafka(kc) => Ok((
                KafkaSourceReader::new(
                    source_name,
                    source_id,
                    worker_id,
                    worker_count,
                    consumer_activator,
                    kc,
                    restored_offsets,
                    logger,
                ),
                None,
            )),
            _ => unreachable!(),
        }
    }
    fn get_next_message(&mut self) -> Result<NextMessage<Self::Key, Self::Value>, anyhow::Error> {
        self.get_next_kafka_message()
    }
}

impl KafkaSourceReader {
    /// Constructor
    pub fn new(
        source_name: String,
        source_id: SourceInstanceId,
        worker_id: usize,
        worker_count: usize,
        consumer_activator: SyncActivator,
        kc: KafkaSourceConnector,
        restored_offsets: Vec<(PartitionId, Option<MzOffset>)>,
        logger: Option<Logger>,
    ) -> KafkaSourceReader {
        let KafkaSourceConnector {
            addrs,
            topic,
            config_options,
            group_id_prefix,
            cluster_id,
            ..
        } = kc;
        let kafka_config = create_kafka_config(
            &source_name,
            &addrs,
            group_id_prefix,
            cluster_id,
            &config_options,
        );
        let (stats_tx, stats_rx) = crossbeam_channel::unbounded();
        let consumer: BaseConsumer<GlueConsumerContext> = kafka_config
            .create_with_context(GlueConsumerContext {
                activator: consumer_activator,
                stats_tx,
            })
            .expect("Failed to create Kafka Consumer");
        let consumer = Arc::new(consumer);

        let mut start_offsets: HashMap<_, _> = kc
            .start_offsets
            .into_iter()
            .map(|(pid, offset)| (pid, offset - 1))
            .collect();

        for (pid, offset) in restored_offsets {
            let pid = match pid {
                PartitionId::Kafka(id) => id,
                _ => panic!("unexpected partition id type"),
            };
            if let Some(offset) = offset {
                if let Some(start_offset) = start_offsets.get_mut(&pid) {
                    *start_offset = std::cmp::max(offset.offset - 1, *start_offset);
                } else {
                    start_offsets.insert(pid, offset.offset - 1);
                }
            }
        }

        let partition_info = Arc::new(Mutex::new(None));
        {
            let partition_info = Arc::downgrade(&partition_info);
            let topic = topic.clone();
            let consumer = Arc::clone(&consumer);
            let metadata_refresh_frequency = config_options
                .get("topic.metadata.refresh.interval.ms")
                // Safe conversion: statement::extract_config enforces that option is a value
                // between 0 and 3600000
                .map(|s| Duration::from_millis(s.parse().unwrap()))
                // Default value obtained from https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md
                .unwrap_or_else(|| Duration::from_secs(300));

            std::thread::Builder::new()
                .name("kafka-metadata".to_string())
                .spawn(move || {
                    while let Some(partition_info) = partition_info.upgrade() {
                        match get_kafka_partitions(&consumer, &topic, Duration::from_secs(30)) {
                            Ok(info) => {
                                *partition_info.lock().unwrap() = Some(info);
                                std::thread::sleep(metadata_refresh_frequency);
                            }
                            Err(_) => std::thread::sleep(Duration::from_secs(30)),
                        }
                    }
                })
                .unwrap();
        }

        KafkaSourceReader {
            topic_name: topic,
            source_name,
            id: source_id,
            partition_consumers: VecDeque::new(),
            consumer,
            worker_id,
            worker_count,
            last_offsets: HashMap::new(),
            start_offsets,
            logger,
            stats_rx,
            last_stats: None,
            partition_info,
        }
    }

    /// Ensures that a partition queue for `pid` exists.
    /// In Kafka, partitions are assigned contiguously. This function consequently
    /// creates partition queues for every p <= pid
    fn add_partition(&mut self, pid: PartitionId) {
        if !crate::source::responsible_for(
            &self.id.source_id,
            self.worker_id,
            self.worker_count,
            &pid,
        ) {
            return;
        }
        let pid = match pid {
            PartitionId::Kafka(p) => p,
            _ => unreachable!(),
        };
        if self.last_offsets.contains_key(&pid) {
            return;
        }

        // Indicate a last offset of -1 if we have not been instructed to have a specific start
        // offset for this topic.
        let start_offset = match self.start_offsets.get(&pid) {
            // Seek to the *next* offset (aka start_offset + 1) that we have not yet processed
            Some(offset) => *offset + 1,
            None => 0,
        };

        self.create_partition_queue(pid, Offset::Offset(start_offset));

        let prev = self.last_offsets.insert(pid, start_offset - 1);

        assert!(prev.is_none());
    }

    /// Returns a count of total number of consumers for this source
    fn get_partition_consumers_count(&self) -> i32 {
        // Note: the number of consumers is guaranteed to always be smaller than
        // expected_partition_count (i32)
        self.partition_consumers.len().try_into().unwrap()
    }

    /// Creates a new partition queue for `partition_id`.
    fn create_partition_queue(&mut self, partition_id: i32, initial_offset: Offset) {
        info!(
            "Activating Kafka queue for {} [{}] (source {}) on worker {}",
            self.topic_name, partition_id, self.id, self.worker_id
        );

        // Collect old partition assignments
        let tpl = self.consumer.assignment().unwrap();
        // Create list from assignments
        let mut partition_list = TopicPartitionList::new();
        for partition in tpl.elements_for_topic(&self.topic_name) {
            partition_list
                .add_partition_offset(partition.topic(), partition.partition(), partition.offset())
                .expect("offset known to be valid");
        }
        // Add new partition
        partition_list
            .add_partition_offset(&self.topic_name, partition_id, initial_offset)
            .expect("offset known to be valid");
        self.consumer
            .assign(&partition_list)
            .expect("assignment known to be valid");

        // Since librdkafka v1.6.0, we need to recreate all partition queues
        // after every call to `self.consumer.assign`.
        let context = Arc::clone(&self.consumer.context());
        for pc in &mut self.partition_consumers {
            pc.partition_queue = self
                .consumer
                .split_partition_queue(&self.topic_name, pc.pid)
                .expect("partition known to be valid");
            pc.partition_queue.set_nonempty_callback({
                let context = Arc::clone(&context);
                move || context.activate()
            });
        }

        let mut partition_queue = self
            .consumer
            .split_partition_queue(&self.topic_name, partition_id)
            .expect("partition known to be valid");
        partition_queue.set_nonempty_callback(move || context.activate());
        self.partition_consumers
            .push_front(PartitionConsumer::new(partition_id, partition_queue));
        assert_eq!(
            self.consumer
                .assignment()
                .unwrap()
                .elements_for_topic(&self.topic_name)
                .len(),
            self.partition_consumers.len()
        );
    }

    /// Fast-forward consumer to specified Kafka Offset. Prints a warning if failed to do so
    /// Assumption: if offset does not exist (for instance, because of compaction), will seek
    /// to the next available offset
    fn fast_forward_consumer(&self, pid: i32, next_offset: i64) {
        let res = self.consumer.seek(
            &self.topic_name,
            pid,
            Offset::Offset(next_offset),
            Duration::from_secs(1),
        );
        match res {
            Ok(_) => {
                let res = self.consumer.position().unwrap_or_default().to_topic_map();
                let position = res
                    .get(&(self.topic_name.clone(), pid))
                    .and_then(|p| match p {
                        Offset::Offset(o) => Some(o),
                        _ => None,
                    });
                if let Some(position) = position {
                    if *position != next_offset {
                        warn!("Did not fast-forward consumer on partition PID: {} to the correct Kafka offset. Currently at offset: {} Expected offset: {}",
                              pid, position, next_offset);
                    } else {
                        info!("Successfully fast-forwarded consumer on partition PID: {} to Kafka offset {}.", pid, position);
                    }
                } else {
                    warn!("Tried to fast-forward consumer on partition PID: {} to Kafka offset {}. Could not obtain new consumer position",
                          pid, next_offset);
                }
            }
            Err(e) => error!(
                "Failed to fast-forward consumer for source:{}, Error:{}",
                self.source_name, e
            ),
        }
    }

    /// This function polls from the next consumer for which a message is available. This function
    /// polls the set round-robin: when a consumer is polled, it is placed at the back of the
    /// queue.
    ///
    /// If a message has an offset that is smaller than the next expected offset for this consumer
    /// (and this partition) we skip this message, and seek to the appropriate offset
    fn get_next_kafka_message(
        &mut self,
    ) -> Result<NextMessage<Option<Vec<u8>>, Option<Vec<u8>>>, anyhow::Error> {
        let partition_info = self.partition_info.lock().unwrap().take();
        if let Some(partitions) = partition_info {
            for pid in partitions {
                self.add_partition(PartitionId::Kafka(pid));
            }
        }
        let mut next_message = NextMessage::Pending;

        // Poll the consumer once. We split the consumer's partitions out into separate queues and
        // poll those individually, but it's still necessary to drive logic that consumes from
        // rdkafka's internal event queue, such as statistics callbacks.
        //
        // Additionally, assigning topics and splitting them off into separate queues is not
        // atomic, so we expect to see at least some messages to show up when polling the consumer
        // directly.
        if let Some(result) = self.consumer.poll(Duration::from_secs(0)) {
            match result {
                Err(e) => error!(
                    "kafka error when polling consumer for source: {} topic: {} : {}",
                    self.source_name, self.topic_name, e
                ),
                Ok(message) => {
                    let source_message = SourceMessage::from(&message);
                    next_message = self.handle_message(source_message);
                }
            }
        }

        self.update_stats();

        let consumer_count = self.get_partition_consumers_count();
        let mut attempts = 0;
        while attempts < consumer_count {
            // First, see if we have a message aleady, either from polling the consumer, above, or
            // from polling the partition queues below.
            if let NextMessage::Ready(_) = next_message {
                // Found a message, exit the loop and return message
                break;
            }

            let message = self.poll_from_next_queue();
            attempts += 1;

            if let Some(message) = message {
                next_message = self.handle_message(message);
            }
        }

        Ok(next_message)
    }

    /// Read any statistics JSON blobs generated via the rdkafka statistics callback.
    fn update_stats(&mut self) {
        while let Ok(stats) = self.stats_rx.try_recv() {
            if let Some(logger) = self.logger.as_mut() {
                logger.log(MaterializedEvent::KafkaSourceStatistics {
                    source_id: self.id,
                    old: self.last_stats.take(),
                    new: Some(stats.clone()),
                });
                self.last_stats = Some(stats);
            }
        }
    }

    /// Polls from the next partition queue and returns the message, if any.
    ///
    /// We maintain the list of partition queues in a queue, and add queues that we polled from to
    /// the end of the queue. We thus swing through all available partition queues in a somewhat
    /// fair manner.
    fn poll_from_next_queue(&mut self) -> Option<SourceMessage<Option<Vec<u8>>, Option<Vec<u8>>>> {
        let mut partition_queue = self.partition_consumers.pop_front().unwrap();

        let message = match partition_queue.get_next_message() {
            Err(e) => {
                let pid = partition_queue.pid();
                let last_offset = self
                    .last_offsets
                    .get(&pid)
                    .expect("partition known to be installed");

                error!(
                        "kafka error consuming from source: {} topic: {}: partition: {} last processed offset: {} : {}",
                        self.source_name,
                        self.topic_name,
                        pid,
                        last_offset,
                        e
                    );
                None
            }
            Ok(m) => m,
        };

        self.partition_consumers.push_back(partition_queue);

        message
    }

    /// Checks if the given message is viable for emission. This checks if the message offset is
    /// past the expected offset and seeks the consumer if it is not.
    fn handle_message(
        &mut self,
        message: SourceMessage<Option<Vec<u8>>, Option<Vec<u8>>>,
    ) -> NextMessage<Option<Vec<u8>>, Option<Vec<u8>>> {
        let partition = match message.partition {
            PartitionId::Kafka(pid) => pid,
            _ => unreachable!(),
        };

        // Convert the received offset back from a 1-indexed MzOffset to the correct offset.
        let offset = message.offset.offset - 1;
        // Offsets are guaranteed to be 1) monotonically increasing *unless* there is
        // a network issue or a new partition added, at which point the consumer may
        // start processing the topic from the beginning, or we may see duplicate offsets
        // At all times, the guarantee : if we see offset x, we have seen all offsets [0,x-1]
        // that we are ever going to see holds.
        // Offsets are guaranteed to be contiguous when compaction is disabled. If compaction
        // is enabled, there may be gaps in the sequence.
        // If we see an "old" offset, we ast-forward the consumer and skip that message

        // Given the explicit consumer to partition assignment, we should never receive a message
        // for a partition for which we have no metadata
        assert!(self.last_offsets.contains_key(&partition));

        let last_offset_ref = self
            .last_offsets
            .get_mut(&partition)
            .expect("partition known to be installed");

        let last_offset = *last_offset_ref;
        if offset <= last_offset {
            info!(
                "Kafka message before expected offset, skipping: \
                             source {} (reading topic {}, partition {}) \
                             received offset {} expected offset {:?}",
                self.source_name,
                self.topic_name,
                partition,
                offset,
                last_offset + 1,
            );
            // Seek to the *next* offset (aka last_offset + 1) that we have not yet processed
            self.fast_forward_consumer(partition, last_offset + 1);
            // We explicitly should not consume the message as we have already processed it
            // However, we make sure to activate the source to make sure that we get a chance
            // to read from this consumer again (even if no new data arrives)
            NextMessage::TransientDelay
        } else {
            *last_offset_ref = offset;
            NextMessage::Ready(message)
        }
    }
}

impl Drop for KafkaSourceReader {
    fn drop(&mut self) {
        // Retract any metrics logged for this source.
        if let Some(logger) = self.logger.as_mut() {
            logger.log(MaterializedEvent::KafkaSourceStatistics {
                source_id: self.id,
                old: self.last_stats.take(),
                new: None,
            });
        }
    }
}

/// Creates a Kafka config.
fn create_kafka_config(
    name: &str,
    addrs: &KafkaAddrs,
    group_id_prefix: Option<String>,
    cluster_id: Uuid,
    config_options: &BTreeMap<String, String>,
) -> ClientConfig {
    let mut kafka_config = ClientConfig::new();

    // Broker configuration.
    kafka_config.set("bootstrap.servers", &addrs.to_string());

    // Default to disabling Kafka auto commit. This can be explicitly enabled
    // by the user if they want to use it for progress tracking.
    kafka_config.set("enable.auto.commit", "false");

    // Always begin ingest at 0 when restarted, even if Kafka contains committed
    // consumer read offsets
    kafka_config.set("auto.offset.reset", "earliest");

    // How often to refresh metadata from the Kafka broker. This can have a
    // minor impact on startup latency and latency after adding a new partition,
    // as the metadata for a partition must be fetched before we can retrieve
    // data from it. We try to manually trigger metadata fetches when it makes
    // sense, but if those manual fetches fail, this is the interval at which we
    // retry.
    //
    // 30s may seem low, but the default is 5m. More frequent metadata refresh
    // rates are surprising to Kafka users, as topic partition counts hardly
    // ever change in production.
    kafka_config.set("topic.metadata.refresh.interval.ms", "30000"); // 30 seconds

    kafka_config.set("fetch.message.max.bytes", "134217728");

    // Consumer group ID. librdkafka requires this, and we use offset commiting
    // to provide a way for users to monitor ingest progress (though we do not
    // rely on the committed offsets for any functionality)
    //
    // This is partially dictated by the user and partially dictated by us.
    // Users can set a prefix so they can see which consumers belong to which
    // Materialize deployment, but we set a suffix to ensure uniqueness. A
    // unique consumer group ID is the most surefire way to ensure that
    // librdkafka does not try to perform its own consumer group balancing,
    // which would wreak havoc with our careful partition assignment strategy.
    kafka_config.set(
        "group.id",
        &format!(
            "{}materialize-{}-{}",
            group_id_prefix.unwrap_or_else(String::new),
            cluster_id,
            name
        ),
    );

    // Patch the librdkafka debug log system into the Rust `log` ecosystem.
    // This is a very simple integration at the moment; enabling `debug`-level
    // logs for the `librdkafka` target enables the full firehouse of librdkafka
    // debug logs. We may want to investigate finer-grained control.
    // TODO(guswynn): replace this when https://github.com/tokio-rs/tracing/pull/1821 is merged
    if log::log_enabled!(target: "librdkafka", log::Level::Debug) {
        debug!("Enabling 'debug' for rdkafka");
        kafka_config.set("debug", "all");
    }

    // Set additional configuration operations from the user. While these look
    // arbitrary, other layers of the system tightly control which configuration
    // options are allowable.
    for (k, v) in config_options {
        kafka_config.set(k, v);
    }

    kafka_config
}

impl<'a> From<&BorrowedMessage<'a>> for SourceMessage<Option<Vec<u8>>, Option<Vec<u8>>> {
    fn from(msg: &BorrowedMessage<'a>) -> Self {
        let kafka_offset = KafkaOffset {
            offset: msg.offset(),
        };
        Self {
            partition: PartitionId::Kafka(msg.partition()),
            offset: kafka_offset.into(),
            upstream_time_millis: msg.timestamp().to_millis(),
            key: msg.key().map(|k| k.to_vec()),
            value: msg.payload().map(|p| p.to_vec()),
        }
    }
}

/// Wrapper around a partition containing the underlying consumer
struct PartitionConsumer {
    /// the partition id with which this consumer is associated
    pid: i32,
    /// The underlying Kafka partition queue
    partition_queue: PartitionQueue<GlueConsumerContext>,
}

impl PartitionConsumer {
    /// Creates a new partition consumer from underlying Kafka consumer
    fn new(pid: i32, partition_queue: PartitionQueue<GlueConsumerContext>) -> Self {
        PartitionConsumer {
            pid,
            partition_queue,
        }
    }

    /// Returns the next message to process for this partition (if any).
    fn get_next_message(
        &mut self,
    ) -> Result<Option<SourceMessage<Option<Vec<u8>>, Option<Vec<u8>>>>, KafkaError> {
        match self.partition_queue.poll(Duration::from_millis(0)) {
            Some(Ok(msg)) => {
                let result = SourceMessage::from(&msg);
                assert_eq!(result.partition, PartitionId::Kafka(self.pid));
                Ok(Some(result))
            }
            Some(Err(err)) => Err(err),
            _ => Ok(None),
        }
    }

    /// Return the partition id for this PartitionConsumer
    fn pid(&self) -> i32 {
        self.pid
    }
}

/// An implementation of [`ConsumerContext`] that forwards statistics to the
/// worker
struct GlueConsumerContext {
    activator: SyncActivator,
    stats_tx: crossbeam_channel::Sender<Jsonb>,
}

impl ClientContext for GlueConsumerContext {
    fn stats_raw(&self, statistics: &[u8]) {
        match Jsonb::from_slice(statistics) {
            Ok(statistics) => {
                self.stats_tx
                    .send(statistics)
                    .expect("timely operator hung up while Kafka source active");
                self.activate();
            }
            Err(e) => error!("failed decoding librdkafka statistics JSON: {}", e),
        };
    }

    // The shape of the rdkafka *Context traits require us to forward to the `MzClientContext`
    // implementation.
    fn log(&self, level: rdkafka::config::RDKafkaLogLevel, fac: &str, log_message: &str) {
        MzClientContext.log(level, fac, log_message)
    }
    fn error(&self, error: rdkafka::error::KafkaError, reason: &str) {
        MzClientContext.error(error, reason)
    }
}

impl GlueConsumerContext {
    fn activate(&self) {
        self.activator
            .activate()
            .expect("timely operator hung up while Kafka source active");
    }
}

impl ConsumerContext for GlueConsumerContext {}

/// Return the list of partition ids associated with a specific topic
fn get_kafka_partitions(
    consumer: &BaseConsumer<GlueConsumerContext>,
    topic: &str,
    timeout: Duration,
) -> Result<Vec<i32>, anyhow::Error> {
    let metadata = consumer.fetch_metadata(Some(topic), timeout)?;
    Ok(metadata.topics()[0]
        .partitions()
        .iter()
        .map(|x| x.id())
        .collect())
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;
    use std::time::Duration;

    use rdkafka::consumer::{BaseConsumer, Consumer};
    use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList};
    use uuid::Uuid;

    // Splitting off a partition queue with an `Offset` that is not `Offset::Beginning` seems to
    // lead to a race condition where sometimes we receive messages from polling the main consumer
    // instead of on the partition queue. This can be surfaced by running the test in a loop (in
    // the dataflow directory) using:
    //
    // cargo stress --lib --release source::kafka::tests::reproduce_kafka_queue_issue
    //
    // cargo-stress can be installed via `cargo install cargo-stress`
    //
    // You need to set up a topic "queue-test" with 1000 "hello" messages in it. Obviously, running
    // this test requires a running Kafka instance at localhost:9092.
    #[test]
    #[ignore]
    fn demonstrate_kafka_queue_race_condition() -> Result<(), anyhow::Error> {
        let topic_name = "queue-test";
        let pid = 0;

        let mut kafka_config = ClientConfig::new();
        kafka_config.set("bootstrap.servers", "localhost:9092".to_string());
        kafka_config.set("enable.auto.commit", "false");
        kafka_config.set("group.id", Uuid::new_v4().to_string());
        kafka_config.set("fetch.message.max.bytes", "100");
        let consumer: BaseConsumer<_> = kafka_config.create()?;

        let consumer = Arc::new(consumer);

        let mut partition_list = TopicPartitionList::new();
        // Using Offset:Beginning here will work fine, only Offset:Offset(0) leads to the race
        // condition.
        partition_list.add_partition_offset(topic_name, pid, Offset::Offset(0))?;

        consumer.assign(&partition_list)?;

        let partition_queue = consumer
            .split_partition_queue(topic_name, pid)
            .expect("missing partition queue");

        let expected_messages = 1_000;

        let mut common_queue_count = 0;
        let mut partition_queue_count = 0;

        loop {
            if let Some(msg) = consumer.poll(Duration::from_millis(0)) {
                match msg {
                    Ok(msg) => {
                        let _payload =
                            std::str::from_utf8(msg.payload().expect("missing payload"))?;
                        if partition_queue_count > 0 {
                            anyhow::bail!("Got message from common queue after we internally switched to partition queue.");
                        }

                        common_queue_count += 1;
                    }
                    Err(err) => anyhow::bail!("{}", err),
                }
            }

            match partition_queue.poll(Duration::from_millis(0)) {
                Some(Ok(msg)) => {
                    let _payload = std::str::from_utf8(msg.payload().expect("missing payload"))?;
                    partition_queue_count += 1;
                }
                Some(Err(err)) => anyhow::bail!("{}", err),
                _ => (),
            }

            if (common_queue_count + partition_queue_count) == expected_messages {
                break;
            }
        }

        assert!(
            common_queue_count == 0,
            "Got {} out of {} messages from common queue. Partition queue: {}",
            common_queue_count,
            expected_messages,
            partition_queue_count
        );

        Ok(())
    }
}