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
use rdkafka::{
    config::ClientConfig,
    consumer::{CommitMode, Consumer, MessageStream as RawMessageStream},
    message::BorrowedMessage as RawMessage,
    Message as KafkaMessageTrait, Offset, TopicPartitionList,
};
use sea_streamer_runtime::spawn_blocking;
use std::{collections::HashSet, fmt::Debug, time::Duration};

use sea_streamer_types::{
    export::{
        async_trait,
        futures::{
            future::Map,
            stream::{Map as StreamMap, StreamFuture},
            FutureExt, StreamExt,
        },
    },
    runtime_error, Consumer as ConsumerTrait, ConsumerGroup, ConsumerMode, ConsumerOptions,
    Message, Payload, SeqNo, SeqPos, ShardId, StreamErr, StreamKey, StreamerUri, Timestamp,
};

use crate::{
    cluster_uri, impl_into_string, stream_err, BaseOptionKey, KafkaConnectOptions, KafkaErr,
    KafkaResult, DEFAULT_TIMEOUT,
};

pub struct KafkaConsumer {
    mode: ConsumerMode,
    inner: Option<RawConsumer>,
    streams: Vec<(StreamKey, ShardId)>,
}

/// rdkafka's StreamConsumer type
pub type RawConsumer = rdkafka::consumer::StreamConsumer<
    rdkafka::consumer::DefaultConsumerContext,
    crate::KafkaAsyncRuntime,
>;

#[repr(transparent)]
pub struct KafkaMessage<'a>(RawMessage<'a>);

const ZERO: ShardId = ShardId::new(0);

#[derive(Debug, Default, Clone)]
pub struct KafkaConsumerOptions {
    mode: ConsumerMode,
    group_id: Option<ConsumerGroup>,
    session_timeout: Option<Duration>,
    auto_offset_reset: Option<AutoOffsetReset>,
    enable_auto_commit: Option<bool>,
    auto_commit_interval: Option<Duration>,
    enable_auto_offset_store: Option<bool>,
    custom_options: Vec<(String, String)>,
}

#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum KafkaConsumerOptionKey {
    GroupId,
    SessionTimeout,
    AutoOffsetReset,
    EnableAutoCommit,
    AutoCommitInterval,
    EnableAutoOffsetStore,
}

type OptionKey = KafkaConsumerOptionKey;

#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum AutoOffsetReset {
    /// Automatically reset the offset to the earliest offset.
    Earliest,
    /// Automatically reset the offset to the latest offset.
    Latest,
    /// Throw exception to the consumer if no previous offset is found for the consumer's group.
    NoReset,
}

/// Concrete type of Future that will yield the next message.
pub type NextFuture<'a> = Map<
    StreamFuture<RawMessageStream<'a>>,
    fn(
        (
            Option<Result<RawMessage<'a>, KafkaErr>>,
            RawMessageStream<'a>,
        ),
    ) -> KafkaResult<KafkaMessage<'a>>,
>;

/// Concrete type of Stream that will yield the next messages.
pub type KafkaMessageStream<'a> = StreamMap<
    RawMessageStream<'a>,
    fn(Result<RawMessage<'a>, KafkaErr>) -> KafkaResult<KafkaMessage<'a>>,
>;

impl KafkaConsumerOptions {
    /// A unique string that identifies the consumer group this consumer belongs to.
    /// This property is required if the consumer uses either the group management functionality
    /// by using subscribe(topic) or the Kafka-based offset management strategy.
    ///
    /// <https://kafka.apache.org/documentation/#connectconfigs_group.id>
    pub fn set_group_id(&mut self, id: ConsumerGroup) -> &mut Self {
        self.group_id = Some(id);
        self
    }
    pub fn group_id(&self) -> Option<&ConsumerGroup> {
        self.group_id.as_ref()
    }

    /// The timeout used to detect worker failures. The worker sends periodic heartbeats
    /// to indicate its liveness to the broker. If no heartbeats are received by the broker
    /// before the expiration of this session timeout, then the broker will remove the worker
    /// from the group and initiate a rebalance.
    ///
    /// <https://kafka.apache.org/documentation/#connectconfigs_session.timeout.ms>
    pub fn set_session_timeout(&mut self, v: Duration) -> &mut Self {
        self.session_timeout = Some(v);
        self
    }
    pub fn session_timeout(&self) -> Option<&Duration> {
        self.session_timeout.as_ref()
    }

    /// Where to stream from when there is no initial offset in Kafka or if the current offset does
    /// not exist any more on the server.
    ///
    /// If unset, defaults to Latest.
    ///
    /// <https://kafka.apache.org/documentation/#consumerconfigs_auto.offset.reset>
    pub fn set_auto_offset_reset(&mut self, v: AutoOffsetReset) -> &mut Self {
        self.auto_offset_reset = Some(v);
        self
    }
    pub fn auto_offset_reset(&self) -> Option<&AutoOffsetReset> {
        self.auto_offset_reset.as_ref()
    }

    /// If enabled, the consumer's offset will be periodically committed in the background.
    ///
    /// If unset, defaults to true.
    ///
    /// <https://kafka.apache.org/documentation/#consumerconfigs_enable.auto.commit>
    pub fn set_enable_auto_commit(&mut self, v: bool) -> &mut Self {
        self.enable_auto_commit = Some(v);
        self
    }
    pub fn enable_auto_commit(&self) -> Option<&bool> {
        self.enable_auto_commit.as_ref()
    }

    /// The interval for offsets to be auto-committed. If `enable_auto_commit` is set to false,
    /// this will have no effect.
    ///
    /// <https://kafka.apache.org/documentation/#consumerconfigs_auto.commit.interval.ms>
    pub fn set_auto_commit_interval(&mut self, v: Duration) -> &mut Self {
        self.auto_commit_interval = Some(v);
        self
    }
    pub fn auto_commit_interval(&self) -> Option<&Duration> {
        self.auto_commit_interval.as_ref()
    }

    /// If enabled, the consumer's offset will be updated as the messages are *read*. This does
    /// not equate to them being *processed*. So if you want to make sure to commit only what
    /// have been processed, set it to false. You will have to manually update these offsets.
    ///
    /// If unset, defaults to true.
    pub fn set_enable_auto_offset_store(&mut self, v: bool) -> &mut Self {
        self.enable_auto_offset_store = Some(v);
        self
    }
    pub fn enable_auto_offset_store(&self) -> Option<&bool> {
        self.enable_auto_offset_store.as_ref()
    }

    /// Add a custom option. If you have an option you frequently use,
    /// please consider open a PR and add it to above.
    pub fn add_custom_option<K, V>(&mut self, key: K, value: V) -> &mut Self
    where
        K: Into<String>,
        V: Into<String>,
    {
        self.custom_options.push((key.into(), value.into()));
        self
    }
    pub fn custom_options(&self) -> impl Iterator<Item = (&str, &str)> {
        self.custom_options
            .iter()
            .map(|(k, v)| (k.as_str(), v.as_str()))
    }

    fn make_client_config(&self, client_config: &mut ClientConfig) {
        if let Some(group_id) = &self.group_id {
            client_config.set(OptionKey::GroupId, group_id.name());
        } else {
            // https://github.com/edenhill/librdkafka/issues/3261
            // librdkafka always require a group_id even when not joining a consumer group
            // But this is purely a client side issue
            client_config.set(OptionKey::GroupId, "abcdefg");
        }
        if let Some(v) = self.session_timeout {
            client_config.set(OptionKey::SessionTimeout, format!("{}", v.as_millis()));
        }
        if let Some(v) = self.auto_offset_reset {
            client_config.set(OptionKey::AutoOffsetReset, v);
        }
        if let Some(v) = self.enable_auto_commit {
            client_config.set(OptionKey::EnableAutoCommit, v.to_string());
        }
        if let Some(v) = self.auto_commit_interval {
            client_config.set(OptionKey::AutoCommitInterval, format!("{}", v.as_millis()));
        }
        if let Some(v) = self.enable_auto_offset_store {
            client_config.set(OptionKey::EnableAutoOffsetStore, v.to_string());
        }
        for (key, value) in self.custom_options() {
            client_config.set(key, value);
        }
    }
}

impl OptionKey {
    pub fn as_str(&self) -> &'static str {
        match self {
            Self::GroupId => "group.id",
            Self::SessionTimeout => "session.timeout.ms",
            Self::AutoOffsetReset => "auto.offset.reset",
            Self::EnableAutoCommit => "enable.auto.commit",
            Self::AutoCommitInterval => "auto.commit.interval.ms",
            Self::EnableAutoOffsetStore => "enable.auto.offset.store",
        }
    }
}

impl AutoOffsetReset {
    pub fn as_str(&self) -> &'static str {
        match self {
            Self::Earliest => "earliest",
            Self::Latest => "latest",
            Self::NoReset => "none",
        }
    }
}

impl_into_string!(OptionKey);
impl_into_string!(AutoOffsetReset);

impl std::fmt::Debug for KafkaConsumer {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("KafkaConsumer").finish()
    }
}

#[async_trait]
impl ConsumerTrait for KafkaConsumer {
    type Error = KafkaErr;
    type Message<'a> = KafkaMessage<'a>;
    // See we don't actually have to Box these! Looking forward to `type_alias_impl_trait`
    type NextFuture<'a> = NextFuture<'a>;
    type Stream<'a> = KafkaMessageStream<'a>;

    /// Seek all streams to the given point in time, with all partitions assigned.
    ///
    /// # Warning
    ///
    /// This async method is not cancel safe. You must await this future,
    /// and this Consumer will be unusable for any operations until it finishes.
    #[inline]
    async fn seek(&mut self, timestamp: Timestamp) -> KafkaResult<()> {
        self.seek_with_timeout(timestamp, DEFAULT_TIMEOUT).await
    }

    /// Rewind all streams across all assigned partitions.
    /// Call [`Consumer::assign`] to assign a partition beforehand,
    /// or [`KafkaConsumer::reassign_partitions`] to assign all partitions.
    async fn rewind(&mut self, offset: SeqPos) -> KafkaResult<()> {
        let mut tpl = TopicPartitionList::new();

        for (stream, shard) in self.streams.iter() {
            tpl.add_partition_offset(
                stream.name(),
                shard.id() as i32,
                match offset {
                    SeqPos::Beginning => Offset::Beginning,
                    SeqPos::End => Offset::End,
                    SeqPos::At(seq) => Offset::Offset(seq.try_into().expect("u64 out of range")),
                },
            )
            .map_err(stream_err)?;
        }

        self.get().assign(&tpl).map_err(stream_err)?;
        Ok(())
    }

    fn assign(&mut self, (stream, shard): (StreamKey, ShardId)) -> KafkaResult<()> {
        if !self.streams.iter().any(|(s, _)| s == &stream) {
            return Err(StreamErr::StreamKeyNotFound);
        }
        if !self
            .streams
            .iter()
            .any(|(s, t)| (s, t) == (&stream, &shard))
        {
            self.streams.push((stream, shard));
        }
        Ok(())
    }

    fn unassign(&mut self, s: (StreamKey, ShardId)) -> KafkaResult<()> {
        if let Some((i, _)) = self.streams.iter().enumerate().find(|(_, t)| &s == *t) {
            self.streams.remove(i);
            if self.streams.is_empty() {
                Err(StreamErr::StreamKeyEmpty)
            } else {
                Ok(())
            }
        } else {
            Err(StreamErr::StreamKeyNotFound)
        }
    }

    fn next(&self) -> Self::NextFuture<'_> {
        self.get().stream().into_future().map(|(res, _)| match res {
            Some(res) => Self::process(res),
            None => panic!("Kafka stream never ends"),
        })
    }

    /// Note: Kafka stream never ends.
    fn stream<'a, 'b: 'a>(&'b mut self) -> Self::Stream<'a> {
        self.get().stream().map(Self::process)
    }
}

impl KafkaConsumer {
    /// Get the underlying StreamConsumer
    #[inline]
    fn get(&self) -> &RawConsumer {
        self.inner
            .as_ref()
            .expect("Client is still inside an async operation, please await the future")
    }

    /// Borrow the inner KafkaConsumer. Use at your own risk.
    pub fn inner(&mut self) -> &RawConsumer {
        self.get()
    }

    fn process(res: Result<RawMessage, KafkaErr>) -> KafkaResult<KafkaMessage> {
        match res {
            Ok(mess) => Ok(KafkaMessage(mess)),
            Err(err) => Err(StreamErr::Backend(err)),
        }
    }

    /// Stream-shards this consumer has been (manually) assigned to.
    /// Note that since this can be changing due to load-balancing,
    /// only ZERO is assigned by default.
    pub fn stream_shards(&self) -> &[(StreamKey, ShardId)] {
        &self.streams
    }

    #[inline]
    async fn async_func<
        T: Send + 'static,
        F: FnOnce(&RawConsumer) -> Result<T, KafkaErr> + Send + 'static,
    >(
        &mut self,
        func: F,
    ) -> KafkaResult<T> {
        if self.inner.is_none() {
            panic!("An async operation is still in progress.");
        }

        // Sadly, `commit` is a sync function, but we want an async API
        // to await when transaction finishes. To avoid Client being dropped while
        // committing, we transfer the ownership into the handler thread,
        // and we take back the ownership after it finishes.
        // Meanwhile, any Client operation will panic.
        // This constraint should be held by the `&mut` signature of this method,
        // but if someone ignores or discards this future, this Consumer will be broken.
        let client = self.inner.take().unwrap();
        let inner = spawn_blocking(move || match func(&client) {
            Ok(res) => Ok((res, client)),
            Err(err) => Err((err, client)),
        })
        .await
        .map_err(runtime_error)?;

        match inner {
            Ok((res, inner)) => {
                self.inner = Some(inner);
                Ok(res)
            }
            Err((err, inner)) => {
                self.inner = Some(inner);
                Err(stream_err(err))
            }
        }
    }

    /// Fetch the partition list of subscribed topics and assign all partitions.
    pub async fn reassign_partitions(&mut self) -> KafkaResult<()> {
        let current: HashSet<StreamKey> = self.streams.iter().map(|(s, _)| s.clone()).collect();
        let mut streams = Vec::new();
        for stream_key in current {
            let s = stream_key.clone();
            let raw = self
                .async_func(move |c| c.fetch_metadata(Some(s.name()), DEFAULT_TIMEOUT))
                .await?;
            let partitions = raw
                .topics()
                .first()
                .unwrap()
                .partitions()
                .iter()
                .map(|p| p.id() as u64);
            for p in partitions {
                streams.push((stream_key.clone(), ShardId::new(p)));
            }
        }
        if streams.is_empty() {
            return Err(StreamErr::Backend(KafkaErr::Subscription(
                "No partitions found.".to_owned(),
            )));
        }
        self.streams = streams;
        Ok(())
    }

    /// Seek all streams to the given point in time, with all partitions assigned.
    ///
    /// # Warning
    ///
    /// This async method is not cancel safe. You must await this future,
    /// and this Consumer will be unusable for any operations until it finishes.
    async fn seek_with_timeout(
        &mut self,
        timestamp: Timestamp,
        timeout: Duration,
    ) -> KafkaResult<()> {
        self.reassign_partitions().await?;

        let mut tpl = TopicPartitionList::new();

        for (stream, shard) in self.streams.iter() {
            tpl.add_partition_offset(
                stream.name(),
                shard.id() as i32,
                Offset::Offset(
                    (timestamp.unix_timestamp_nanos() / 1_000_000)
                        .try_into()
                        .expect("KafkaConsumer::seek: timestamp out of range"),
                ),
            )
            .map_err(stream_err)?;
        }

        let tpl = self
            .async_func(move |c| c.offsets_for_times(tpl, timeout))
            .await?;

        self.inner
            .as_mut()
            .unwrap()
            .assign(&tpl)
            .map_err(stream_err)?;

        Ok(())
    }

    /// Commit an "ack" to broker for having processed this message.
    ///
    /// # Warning
    ///
    /// This async method is not cancel safe. You must await this future,
    /// and this Consumer will be unusable for any operations until it finishes.
    pub async fn commit_message(&mut self, mess: &KafkaMessage<'_>) -> KafkaResult<()> {
        self.commit(&mess.stream_key(), &mess.shard_id(), &mess.sequence())
            .await
    }

    /// Commit an "ack" to broker for having processed up to this cursor.
    ///
    /// # Warning
    ///
    /// This async method is not cancel safe. You must await this future,
    /// and this Consumer will be unusable for any operations until it finishes.
    pub async fn commit_with(
        &mut self,
        (stream_key, shard_id, sequence): &(StreamKey, ShardId, SeqNo),
    ) -> KafkaResult<()> {
        self.commit(stream_key, shard_id, sequence).await
    }

    /// Commit an "ack" to broker for having processed up to this cursor.
    ///
    /// # Warning
    ///
    /// This async method is not cancel safe. You must await this future,
    /// and this Consumer will be unusable for any operations until it finishes.
    pub async fn commit(
        &mut self,
        stream: &StreamKey,
        shard: &ShardId,
        seq: &SeqNo,
    ) -> KafkaResult<()> {
        if self.mode == ConsumerMode::RealTime {
            return Err(StreamErr::CommitNotAllowed);
        }
        let mut tpl = TopicPartitionList::new();
        tpl.add_partition_offset(
            stream.name(),
            shard.id() as i32,
            Offset::Offset((*seq).try_into().expect("u64 out of range")),
        )
        .map_err(stream_err)?;

        self.async_func(move |c| c.commit(&tpl, CommitMode::Sync))
            .await
    }

    /// Store the offset so that it will be committed.
    /// You must have `set_enable_auto_offset_store` to false.
    pub fn store_offset(
        &mut self,
        stream: &StreamKey,
        shard: &ShardId,
        seq: &SeqNo,
    ) -> KafkaResult<()> {
        self.get()
            .store_offset(
                stream.name(),
                shard.id() as i32,
                (*seq).try_into().expect("u64 out of range"),
            )
            .map_err(stream_err)
    }

    /// Store the offset for this message so that it will be committed.
    /// You must have `set_enable_auto_offset_store` to false.
    pub fn store_offset_for_message(&mut self, mess: &KafkaMessage<'_>) -> KafkaResult<()> {
        self.store_offset(&mess.stream_key(), &mess.shard_id(), &mess.sequence())
    }

    /// Store the offset with message identifier so that it will be committed.
    /// You must have `set_enable_auto_offset_store` to false.
    pub fn store_offset_with(
        &mut self,
        (stream_key, shard_id, sequence): &(StreamKey, ShardId, SeqNo),
    ) -> KafkaResult<()> {
        self.store_offset(stream_key, shard_id, sequence)
    }
}

impl<'a> KafkaMessage<'a> {
    fn mess(&self) -> &RawMessage {
        &self.0
    }
}

impl<'a> Debug for KafkaMessage<'a> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        self.mess().fmt(f)
    }
}

impl<'a> Message for KafkaMessage<'a> {
    fn stream_key(&self) -> StreamKey {
        StreamKey::new(self.mess().topic()).expect("A message should carry a valid stream key")
    }

    fn shard_id(&self) -> ShardId {
        ShardId::new(self.mess().partition() as u64)
    }

    fn sequence(&self) -> SeqNo {
        self.mess().offset() as SeqNo
    }

    fn timestamp(&self) -> Timestamp {
        Timestamp::from_unix_timestamp_nanos(
            self.mess()
                .timestamp()
                .to_millis()
                .expect("message.timestamp() is None") as i128
                * 1_000_000,
        )
        .expect("from_unix_timestamp_nanos")
    }

    fn message(&self) -> Payload {
        Payload::new(self.mess().payload().unwrap_or_default())
    }
}

impl ConsumerOptions for KafkaConsumerOptions {
    type Error = KafkaErr;

    fn new(mode: ConsumerMode) -> Self {
        KafkaConsumerOptions {
            mode,
            ..Default::default()
        }
    }

    fn mode(&self) -> KafkaResult<&ConsumerMode> {
        Ok(&self.mode)
    }

    fn consumer_group(&self) -> KafkaResult<&ConsumerGroup> {
        self.group_id.as_ref().ok_or(StreamErr::ConsumerGroupNotSet)
    }

    /// If multiple consumers shares the same group, only one consumer in the group will
    /// receive a message, i.e. it is load-balanced.
    ///
    /// However, the load-balancing mechanism is what makes Kafka different:
    ///
    /// Each stream is divided into multiple shards (known as partition),
    /// and each partition will be assigned to only one consumer in a group.
    ///
    /// Say there are 2 consumers (in the group) and 2 partitions, then each consumer
    /// will receive messages from one partition, and they are thus load-balanced.
    ///
    /// If there are 2 consumers and 3 partitions, then one consumer will be assigned
    /// 2 partitions, and the other will be assigned only 1.
    ///
    /// However if the stream has only 1 partition, even if there are many consumers,
    /// these messages will only be received by the assigned consumer, and other consumers
    /// will be in stand-by mode, resulting in a hot-failover setup.
    fn set_consumer_group(&mut self, group: ConsumerGroup) -> KafkaResult<&mut Self> {
        self.group_id = Some(group);
        Ok(self)
    }
}

pub(crate) fn create_consumer(
    streamer: &StreamerUri,
    base_options: &KafkaConnectOptions,
    options: &KafkaConsumerOptions,
    streams: Vec<StreamKey>,
) -> Result<KafkaConsumer, KafkaErr> {
    let mut client_config = ClientConfig::new();
    client_config.set(BaseOptionKey::BootstrapServers, cluster_uri(streamer)?);
    base_options.make_client_config(&mut client_config);
    options.make_client_config(&mut client_config);

    let consumer: RawConsumer = client_config.create()?;

    if !streams.is_empty() {
        let topics: Vec<&str> = streams.iter().map(|s| s.name()).collect();
        consumer.subscribe(&topics)?;
    } else {
        panic!("no topic?");
    }

    Ok(KafkaConsumer {
        mode: options.mode,
        inner: Some(consumer),
        streams: streams.into_iter().map(|s| (s, ZERO)).collect(),
    })
}