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
use std::sync::Arc;

use anyhow::Result;
use tracing::{debug, error, trace, instrument, info, warn};
use futures_util::stream::{Stream, select_all};
use once_cell::sync::Lazy;
use futures_util::future::{Either, err, join_all};
use futures_util::stream::{StreamExt, once, iter};
use futures_util::FutureExt;

use fluvio_types::PartitionId;
use fluvio_types::defaults::{FLUVIO_CLIENT_MAX_FETCH_BYTES, FLUVIO_MAX_SIZE_TOPIC_NAME};
use fluvio_types::event::offsets::OffsetPublisher;
use fluvio_spu_schema::server::stream_fetch::{
    DefaultStreamFetchRequest, DefaultStreamFetchResponse, CHAIN_SMARTMODULE_API,
};
use fluvio_spu_schema::Isolation;
use fluvio_protocol::record::ReplicaKey;
use fluvio_protocol::link::ErrorCode;
use fluvio_protocol::record::Batch;

use crate::FluvioError;
use crate::metrics::ClientMetrics;
use crate::offset::{Offset, fetch_offsets};
use crate::spu::{SpuDirectory, SpuPool};
use derive_builder::Builder;

pub use fluvio_protocol::record::ConsumerRecord as Record;
pub use fluvio_spu_schema::server::smartmodule::SmartModuleInvocation;
pub use fluvio_spu_schema::server::smartmodule::SmartModuleInvocationWasm;
pub use fluvio_spu_schema::server::smartmodule::SmartModuleKind;
pub use fluvio_spu_schema::server::smartmodule::SmartModuleContextData;
pub use fluvio_smartmodule::dataplane::smartmodule::SmartModuleExtraParams;

/// An interface for consuming events from a particular partition
///
///
///
/// [`Offset`]: struct.Offset.html
/// [`partition_consumer`]: struct.Fluvio.html#method.partition_consumer
/// [`Fluvio`]: struct.Fluvio.html
pub struct PartitionConsumer<P = SpuPool> {
    topic: String,
    partition: PartitionId,
    pool: Arc<P>,
    metrics: Arc<ClientMetrics>,
}

// Manually implement Clone because the derive macro would require the
// generic type to also be Clone, here `P`
impl<P> Clone for PartitionConsumer<P> {
    fn clone(&self) -> Self {
        Self {
            topic: self.topic.clone(),
            partition: self.partition,
            pool: self.pool.clone(),
            metrics: self.metrics.clone(),
        }
    }
}

impl<P> PartitionConsumer<P>
where
    P: SpuDirectory,
{
    pub fn new(
        topic: String,
        partition: PartitionId,
        pool: Arc<P>,
        metrics: Arc<ClientMetrics>,
    ) -> Self {
        Self {
            topic,
            partition,
            pool,
            metrics,
        }
    }

    /// Returns the name of the Topic that this consumer reads from
    pub fn topic(&self) -> &str {
        &self.topic
    }

    /// Returns the ID of the partition that this consumer reads from
    pub fn partition(&self) -> PartitionId {
        self.partition
    }

    /// Return a shared instance of `ClientMetrics`
    pub fn metrics(&self) -> Arc<ClientMetrics> {
        self.metrics.clone()
    }

    /// Continuously streams events from a particular offset in the consumer's partition
    ///
    /// Streaming is one of the two ways to consume events in Fluvio.
    /// It is a continuous request for new records arriving in a partition,
    /// beginning at a particular offset. You specify the starting point of the
    /// stream using an [`Offset`] and periodically receive events, either individually
    /// or in batches.
    ///
    /// Note this uses ConsumerRecord instead of batches
    ///
    /// If you want more fine-grained control over how records are streamed,
    /// check out the [`stream_with_config`] method.
    ///
    /// # Example
    ///
    /// ```
    /// # use fluvio::{PartitionConsumer};
    /// # use fluvio::{Offset, ConsumerConfig};
    /// # mod futures {
    /// #     pub use futures_util::stream::StreamExt;
    /// # }
    /// # async fn example(consumer: &PartitionConsumer) -> anyhow::Result<()> {
    /// use futures::StreamExt;
    /// let mut stream = consumer.stream(Offset::beginning()).await?;
    /// while let Some(Ok(record)) = stream.next().await {
    ///     let key_str = record.get_key().map(|key| key.as_utf8_lossy_string());
    ///     let value_str = record.get_value().as_utf8_lossy_string();
    ///     println!("Got event: key={:?}, value={value_str}", key_str);
    /// }
    /// # Ok(())
    /// # }
    /// ```
    ///
    /// [`Offset`]: struct.Offset.html
    /// [`ConsumerConfig`]: struct.ConsumerConfig.html
    /// [`stream_with_config`]: struct.ConsumerConfig.html#method.stream_with_config
    #[instrument(skip(self, offset))]
    pub async fn stream(
        &self,
        offset: Offset,
    ) -> Result<impl Stream<Item = Result<Record, ErrorCode>>> {
        let config = ConsumerConfig::builder().build()?;
        let stream = self.stream_with_config(offset, config).await?;

        Ok(stream)
    }

    /// Continuously streams events from a particular offset in the consumer's partition
    ///
    /// Most of the time, you shouldn't need to use a custom [`ConsumerConfig`].
    /// If you don't know what these settings do, try checking out the simpler
    /// [`PartitionConsumer::stream`] method that uses the default streaming settings.
    ///
    /// Streaming is one of the two ways to consume events in Fluvio.
    /// It is a continuous request for new records arriving in a partition,
    /// beginning at a particular offset. You specify the starting point of the
    /// stream using an [`Offset`] and a [`ConsumerConfig`], and periodically
    /// receive events, either individually or in batches.
    ///
    /// # Example
    ///
    /// ```
    /// # use fluvio::{PartitionConsumer};
    /// # use fluvio::{Offset, ConsumerConfig};
    /// # mod futures {
    /// #     pub use futures_util::stream::StreamExt;
    /// # }
    /// # async fn example(consumer: &PartitionConsumer) -> anyhow::Result<()> {
    /// use futures::StreamExt;
    /// // Use a custom max_bytes value in the config
    /// let fetch_config = ConsumerConfig::builder()
    ///     .max_bytes(1000)
    ///     .build()?;
    /// let mut stream = consumer.stream_with_config(Offset::beginning(), fetch_config).await?;
    /// while let Some(Ok(record)) = stream.next().await {
    ///     let key_str = record.get_key().map(|key| key.as_utf8_lossy_string());
    ///     let value_str = record.get_value().as_utf8_lossy_string();
    ///     println!("Got record: key={:?}, value={}", key_str, value_str);
    /// }
    /// # Ok(())
    /// # }
    /// ```
    ///
    /// [`Offset`]: struct.Offset.html
    /// [`ConsumerConfig`]: struct.ConsumerConfig.html
    #[instrument(skip(self, offset, config))]
    pub async fn stream_with_config(
        &self,
        offset: Offset,
        config: ConsumerConfig,
    ) -> Result<impl Stream<Item = Result<Record, ErrorCode>>> {
        let (stream, start_offset) = self
            .inner_stream_batches_with_config(offset, config)
            .await?;
        let partition = self.partition;
        let flattened = stream.flat_map(move |result: Result<Batch, _>| match result {
            Err(e) => Either::Right(once(err(e))),
            Ok(batch) => {
                let records =
                    batch
                        .into_consumer_records_iter(partition)
                        .filter_map(move |record| {
                            if record.offset >= start_offset {
                                Some(Ok(record))
                            } else {
                                None
                            }
                        });
                Either::Left(iter(records))
            }
        });

        Ok(flattened)
    }

    /// Continuously streams batches of messages, starting an offset in the consumer's partition
    ///
    /// ```
    /// # use fluvio::{PartitionConsumer};
    /// # use fluvio::{Offset, ConsumerConfig};
    /// # mod futures {
    /// #     pub use futures_util::stream::StreamExt;
    /// # }
    /// # async fn example(consumer: &PartitionConsumer) -> anyhow::Result<()> {
    /// use futures::StreamExt;
    /// // Use a custom max_bytes value in the config
    /// let fetch_config = ConsumerConfig::builder()
    ///     .max_bytes(1000)
    ///     .build()?;
    /// let mut stream = consumer.stream_batches_with_config(Offset::beginning(), fetch_config).await?;
    /// while let Some(Ok(batch)) = stream.next().await {
    ///     for record in batch.records() {
    ///         let key_str = record.key().map(|key| key.as_utf8_lossy_string());
    ///         let value_str = record.value().as_utf8_lossy_string();
    ///         println!("Got record: key={:?}, value={}", key_str, value_str);
    ///     }
    /// }
    /// # Ok(())
    /// # }
    /// ```
    #[instrument(skip(self, offset, config))]
    pub async fn stream_batches_with_config(
        &self,
        offset: Offset,
        config: ConsumerConfig,
    ) -> Result<impl Stream<Item = Result<Batch, ErrorCode>>> {
        let (stream, _start_offset) = self
            .inner_stream_batches_with_config(offset, config)
            .await?;
        Ok(stream)
    }

    /// Continuously streams batches of messages, starting an offset in the consumer's partition
    /// Returns both the stream and the start offset of the stream.
    #[instrument(skip(self, offset, config))]
    async fn inner_stream_batches_with_config(
        &self,
        offset: Offset,
        config: ConsumerConfig,
    ) -> Result<(
        impl Stream<Item = Result<Batch, ErrorCode>>,
        fluvio_protocol::record::Offset,
    )> {
        let (stream, start_offset) = self.request_stream(offset, config).await?;
        let metrics = self.metrics.clone();
        let flattened =
            stream.flat_map(move |batch_result: Result<DefaultStreamFetchResponse, _>| {
                let response = match batch_result {
                    Ok(response) => response,
                    Err(e) => return Either::Right(once(err(e))),
                };

                // If we ever get an error_code AND batches of records, we want to first send
                // the records down the consumer stream, THEN an Err with the error inside.
                // This way the consumer always gets to read all records that were properly
                // processed before hitting an error, so that the error does not obscure those records.

                let inner_metrics = metrics.clone();
                let batches =
                    response
                        .partition
                        .records
                        .batches
                        .into_iter()
                        .map(move |raw_batch| {
                            inner_metrics
                                .consumer()
                                .add_records(raw_batch.records_len() as u64);
                            inner_metrics
                                .consumer()
                                .add_bytes(raw_batch.batch_len() as u64);

                            let batch: Result<Batch, _> = raw_batch.try_into();
                            match batch {
                                Ok(batch) => Ok(batch),
                                Err(err) => Err(ErrorCode::Other(err.to_string())),
                            }
                        });
                let error = {
                    let code = response.partition.error_code;
                    match code {
                        ErrorCode::None => None,
                        _ => Some(Err(code)),
                    }
                };

                let items = batches.chain(error.into_iter());
                Either::Left(iter(items))
            });

        Ok((flattened, start_offset))
    }

    /// Creates a stream of `DefaultStreamFetchResponse` for older consumers who rely
    /// on the internal structure of the fetch response. New clients should use the
    /// `stream` and `stream_with_config` methods.
    /// Returns both the stream and the start offset of the stream.
    #[instrument(skip(self, config))]
    async fn request_stream(
        &self,
        offset: Offset,
        config: ConsumerConfig,
    ) -> Result<(
        impl Stream<Item = Result<DefaultStreamFetchResponse, ErrorCode>>,
        fluvio_protocol::record::Offset,
    )> {
        use fluvio_future::task::spawn;
        use futures_util::stream::empty;

        let replica = ReplicaKey::new(&self.topic, self.partition);
        let mut serial_socket = self.pool.create_serial_socket(&replica).await?;
        let offsets = fetch_offsets(&mut serial_socket, &replica).await?;

        let start_absolute_offset = offset.resolve(&offsets).await?;
        let end_absolute_offset = offsets.last_stable_offset;
        let record_count = end_absolute_offset - start_absolute_offset;

        debug!(start_absolute_offset, end_absolute_offset, record_count);

        let stream_request = DefaultStreamFetchRequest::builder()
            .topic(self.topic.to_owned())
            .partition(self.partition)
            .fetch_offset(start_absolute_offset)
            .isolation(config.isolation)
            .max_bytes(config.max_bytes)
            .smartmodules(config.smartmodule)
            .build()?;

        let stream_fetch_version = serial_socket
            .versions()
            .lookup_version::<DefaultStreamFetchRequest>()
            .unwrap_or(CHAIN_SMARTMODULE_API - 1);
        debug!(%stream_fetch_version, "stream_fetch_version");
        if stream_fetch_version < CHAIN_SMARTMODULE_API {
            warn!("SPU does not support SmartModule chaining. SmartModules will not be applied to the stream");
        }

        let mut stream = self
            .pool
            .create_stream_with_version(&replica, stream_request, stream_fetch_version)
            .await?;

        let ft_stream = async move {
            if let Some(Ok(raw_response)) = stream.next().await {
                let response: DefaultStreamFetchResponse = raw_response;

                let stream_id = response.stream_id;

                trace!("first stream response: {:#?}", response);
                debug!(
                    stream_id,
                    last_offset = ?response.partition.next_offset_for_fetch(),
                    "first stream response"
                );

                let publisher = OffsetPublisher::shared(0);
                let mut listener = publisher.change_listener();

                // update stream with received offsets
                spawn(async move {
                    use fluvio_spu_schema::server::update_offset::{UpdateOffsetsRequest, OffsetUpdate};

                    loop {
                        let fetch_last_value = listener.listen().await;
                        debug!(fetch_last_value, stream_id, "received end fetch");
                        if fetch_last_value < 0 {
                            info!("fetch last is end, terminating");
                            break;
                        } else {
                            debug!(
                                offset = fetch_last_value,
                                session_id = stream_id,
                                "sending back offset to spu"
                            );
                            let request = UpdateOffsetsRequest {
                                offsets: vec![OffsetUpdate {
                                    offset: fetch_last_value,
                                    session_id: stream_id,
                                }],
                            };
                            debug!(?request, "Sending offset update request:");
                            let response = serial_socket.send_receive(request).await;
                            if let Err(err) = response {
                                error!("error sending offset: {:#?}", err);
                                break;
                            }
                        }
                    }
                    debug!(stream_id, "offset fetch update loop end");
                });

                // send back first offset records exists
                if let Some(last_offset) = response.partition.next_offset_for_fetch() {
                    debug!(last_offset, "notify new last offset");
                    publisher.update(last_offset);
                }

                let response_publisher = publisher.clone();
                let update_stream = StreamExt::map(stream, move |item| {
                    item.map(|response| {
                        if let Some(last_offset) = response.partition.next_offset_for_fetch() {
                            debug!(last_offset, stream_id, "received last offset from spu");
                            response_publisher.update(last_offset);
                        }
                        response
                    })
                    .map_err(|e| {
                        error!(?e, "error in stream");
                        ErrorCode::Other(e.to_string())
                    })
                });
                Either::Left(
                    iter(vec![Ok(response)])
                        .chain(publish_stream::EndPublishSt::new(update_stream, publisher)),
                )
            } else {
                info!("stream ended");
                Either::Right(empty())
            }
        };

        let stream = if config.disable_continuous {
            TakeRecords::new(ft_stream.flatten_stream().boxed(), record_count).boxed()
        } else {
            ft_stream.flatten_stream().boxed()
        };

        Ok((stream, start_absolute_offset))
    }
}

/// Wrap an inner record stream and only stream until a given number of records have been fetched.
///
/// This is used for "disable continuous" mode. In this mode, we first make a FetchOffsetPartitionResponse
/// in order to see the starting and ending offsets currently available for this partition.
/// Based on the starting offset the caller asks for, we can figure out the "record count", or
/// how many records from the start onward we know for sure we can stream without waiting.
/// We then use `TakeRecords` to stop the stream as soon as we reach that point, so the user
/// (e.g. on the CLI) does not spend any time waiting for new records to be produced, they are
/// simply given all the records that are already available.
struct TakeRecords<S> {
    remaining: i64,
    stream: S,
}

impl<S> TakeRecords<S>
where
    S: Stream<Item = Result<DefaultStreamFetchResponse, ErrorCode>> + std::marker::Unpin,
{
    pub fn new(stream: S, until: i64) -> Self {
        Self {
            remaining: until,
            stream,
        }
    }
}

impl<S> Stream for TakeRecords<S>
where
    S: Stream<Item = Result<DefaultStreamFetchResponse, ErrorCode>> + std::marker::Unpin,
{
    type Item = S::Item;

    fn poll_next(
        mut self: std::pin::Pin<&mut Self>,
        cx: &mut std::task::Context<'_>,
    ) -> std::task::Poll<Option<Self::Item>> {
        use std::{pin::Pin, task::Poll};
        use futures_util::ready;
        if self.remaining <= 0 {
            return Poll::Ready(None);
        }
        let next = ready!(Pin::new(&mut self.as_mut().stream).poll_next(cx));
        match next {
            Some(Ok(response)) => {
                // Count how many records are present in this batch's response
                let count: usize = response
                    .partition
                    .records
                    .batches
                    .iter()
                    .map(|it| it.records_len())
                    .sum();
                let diff = self.remaining - count as i64;
                self.remaining = diff.max(0);
                Poll::Ready(Some(Ok(response)))
            }
            other => Poll::Ready(other),
        }
    }
}

mod publish_stream {

    use std::pin::Pin;
    use std::sync::Arc;
    use std::task::{Poll, Context};

    use pin_project::pin_project;
    use futures_util::ready;

    use super::Stream;
    use super::OffsetPublisher;

    // signal offset when stream is done
    #[pin_project]
    pub struct EndPublishSt<St> {
        #[pin]
        stream: St,
        publisher: Arc<OffsetPublisher>,
    }

    impl<St> EndPublishSt<St> {
        pub fn new(stream: St, publisher: Arc<OffsetPublisher>) -> Self {
            Self { stream, publisher }
        }
    }

    impl<S: Stream> Stream for EndPublishSt<S> {
        type Item = S::Item;

        fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<S::Item>> {
            let this = self.project();

            let item = ready!(this.stream.poll_next(cx));
            if item.is_none() {
                this.publisher.update(-1);
            }
            Poll::Ready(item)
        }

        fn size_hint(&self) -> (usize, Option<usize>) {
            self.stream.size_hint()
        }
    }
}

/// MAX FETCH BYTES
static MAX_FETCH_BYTES: Lazy<i32> = Lazy::new(|| {
    use std::env;
    use fluvio_protocol::Encoder;
    use fluvio_spu_schema::fetch::FetchResponse;
    use fluvio_spu_schema::fetch::FetchableTopicResponse;
    use fluvio_spu_schema::fetch::FetchablePartitionResponse;

    use fluvio_protocol::record::MemoryRecords;

    let var_value = env::var("FLV_CLIENT_MAX_FETCH_BYTES").unwrap_or_default();
    let max_bytes: i32 = var_value.parse().unwrap_or_else(|_| {
        FetchResponse::<MemoryRecords>::default().write_size(0) as i32
            + FetchableTopicResponse::<MemoryRecords>::default().write_size(0) as i32
            + FetchablePartitionResponse::<MemoryRecords>::default().write_size(0) as i32
            + FLUVIO_MAX_SIZE_TOPIC_NAME as i32 // using max size of topic name
            + FLUVIO_CLIENT_MAX_FETCH_BYTES
    });
    max_bytes
});

/// Configures the behavior of consumer fetching and streaming
#[derive(Debug, Builder, Clone)]
#[builder(build_fn(private, name = "build_impl"))]
pub struct ConsumerConfig {
    #[builder(default)]
    disable_continuous: bool,
    #[builder(default = "*MAX_FETCH_BYTES")]
    pub max_bytes: i32,
    #[builder(default)]
    pub isolation: Isolation,
    #[builder(default)]
    pub(crate) smartmodule: Vec<SmartModuleInvocation>,
}

impl ConsumerConfig {
    pub fn builder() -> ConsumerConfigBuilder {
        ConsumerConfigBuilder::default()
    }
}

impl ConsumerConfigBuilder {
    pub fn build(&self) -> Result<ConsumerConfig> {
        let config = self.build_impl().map_err(|e| {
            FluvioError::ConsumerConfig(format!("Missing required config option: {e}"))
        })?;
        Ok(config)
    }
}

/// Strategy used to select which partitions and from which topics should be streamed by the [`MultiplePartitionConsumer`]
#[derive(Clone)]
pub enum PartitionSelectionStrategy {
    /// Consume from all the partitions of a given topic
    All(String),
    /// Consume from a given list of topics and partitions
    Multiple(Vec<(String, PartitionId)>),
}

impl PartitionSelectionStrategy {
    async fn selection(&self, spu_pool: Arc<SpuPool>) -> Result<Vec<(String, PartitionId)>> {
        let pairs = match self {
            PartitionSelectionStrategy::All(topic) => {
                let topics = spu_pool.metadata.topics();
                let topic_spec = topics
                    .lookup_by_key(topic)
                    .await?
                    .ok_or_else(|| FluvioError::TopicNotFound(topic.to_string()))?
                    .spec;
                let partition_count = topic_spec.partitions();
                (0..(partition_count as PartitionId))
                    .map(|partition| (topic.clone(), partition))
                    .collect::<Vec<_>>()
            }
            PartitionSelectionStrategy::Multiple(topic_partition) => topic_partition.to_owned(),
        };
        Ok(pairs)
    }
}
#[derive(Clone)]
pub struct MultiplePartitionConsumer {
    strategy: PartitionSelectionStrategy,
    pool: Arc<SpuPool>,
    metrics: Arc<ClientMetrics>,
}

impl MultiplePartitionConsumer {
    pub(crate) fn new(
        strategy: PartitionSelectionStrategy,
        pool: Arc<SpuPool>,
        metrics: Arc<ClientMetrics>,
    ) -> Self {
        Self {
            strategy,
            pool,
            metrics,
        }
    }

    /// Continuously streams events from a particular offset in the selected partitions
    ///
    /// Streaming is one of the two ways to consume events in Fluvio.
    /// It is a continuous request for new records arriving in the selected partitions,
    /// beginning at a particular offset. You specify the starting point of the
    /// stream using an [`Offset`] and periodically receive events, either individually
    /// or in batches.
    ///
    /// If you want more fine-grained control over how records are streamed,
    /// check out the [`stream_with_config`] method.
    ///
    /// # Example
    ///
    /// ```
    /// # use fluvio::{MultiplePartitionConsumer};
    /// # use fluvio::{Offset, ConsumerConfig};
    /// # mod futures {
    /// #     pub use futures_util::stream::StreamExt;
    /// # }
    /// # async fn example(consumer: &MultiplePartitionConsumer) -> anyhow::Result<()> {
    /// use futures::StreamExt;
    /// let mut stream = consumer.stream(Offset::beginning()).await?;
    /// while let Some(Ok(record)) = stream.next().await {
    ///     let key_str = record.get_key().map(|key| key.as_utf8_lossy_string());
    ///     let value_str = record.get_value().as_utf8_lossy_string();
    ///     println!("Got event: key={:?}, value={}", key_str, value_str);
    /// }
    /// # Ok(())
    /// # }
    /// ```
    ///
    /// [`Offset`]: struct.Offset.html
    /// [`ConsumerConfig`]: struct.ConsumerConfig.html
    /// [`stream_with_config`]: struct.ConsumerConfig.html#method.stream_with_config
    #[instrument(skip(self, offset))]
    pub async fn stream(
        &self,
        offset: Offset,
    ) -> Result<impl Stream<Item = Result<Record, ErrorCode>>> {
        let config = ConsumerConfig::builder().build()?;
        let stream = self.stream_with_config(offset, config).await?;

        Ok(stream)
    }

    /// Continuously streams events from a particular offset in the selected partitions
    ///
    /// Most of the time, you shouldn't need to use a custom [`ConsumerConfig`].
    /// If you don't know what these settings do, try checking out the simpler
    /// [`stream`] method that uses the default streaming settings.
    ///
    /// Streaming is one of the two ways to consume events in Fluvio.
    /// It is a continuous request for new records arriving in the selected partitions,
    /// beginning at a particular offset. You specify the starting point of the
    /// stream using an [`Offset`] and a [`ConsumerConfig`], and periodically
    /// receive events, either individually or in batches.
    ///
    /// # Example
    ///
    /// ```
    /// # use fluvio::{MultiplePartitionConsumer};
    /// # use fluvio::{Offset, ConsumerConfig};
    /// # mod futures {
    /// #     pub use futures_util::stream::StreamExt;
    /// # }
    /// # async fn example(consumer: &MultiplePartitionConsumer) -> anyhow::Result<()> {
    /// use futures::StreamExt;
    /// // Use a custom max_bytes value in the config
    /// let fetch_config = ConsumerConfig::builder()
    ///     .max_bytes(1000)
    ///     .build()?;
    /// let mut stream = consumer.stream_with_config(Offset::beginning(), fetch_config).await?;
    /// while let Some(Ok(record)) = stream.next().await {
    ///     let key_str = record.get_key().map(|key| key.as_utf8_lossy_string());
    ///     let value_str = record.get_value().as_utf8_lossy_string();
    ///     println!("Got record: key={:?}, value={}", key_str, value_str);
    /// }
    /// # Ok(())
    /// # }
    /// ```
    ///
    /// [`Offset`]: struct.Offset.html
    /// [`ConsumerConfig`]: struct.ConsumerConfig.html
    #[instrument(skip(self, offset, config))]
    pub async fn stream_with_config(
        &self,
        offset: Offset,
        config: ConsumerConfig,
    ) -> Result<impl Stream<Item = Result<Record, ErrorCode>>> {
        let consumers = self
            .strategy
            .selection(self.pool.clone())
            .await?
            .into_iter()
            .map(|(topic, partition)| {
                PartitionConsumer::new(
                    topic,
                    partition as PartitionId,
                    self.pool.clone(),
                    self.metrics.clone(),
                )
            })
            .collect::<Vec<_>>();

        let streams_future = consumers
            .iter()
            .map(|consumer| consumer.stream_with_config(offset.clone(), config.clone()));

        let streams_result = join_all(streams_future).await;

        let streams = streams_result.into_iter().collect::<Result<Vec<_>, _>>()?;

        Ok(select_all(streams))
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn test_consumer_config_default() {
        let _config = ConsumerConfig::builder().build().unwrap();
    }
}