rskafka 0.6.0

A minimal Rust client for Apache Kafka
Documentation
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
//! Building blocks for more advanced consumer chains.
//!
//! # Usage
//! ```no_run
//! # async fn test() {
//! use futures::StreamExt;
//! use rskafka::client::{
//!     ClientBuilder,
//!     consumer::{
//!         StartOffset,
//!         StreamConsumerBuilder,
//!     },
//!     partition::UnknownTopicHandling,
//! };
//! use std::sync::Arc;
//!
//! // get partition client
//! let connection = "localhost:9093".to_owned();
//! let client = ClientBuilder::new(vec![connection]).build().await.unwrap();
//! let partition_client = Arc::new(
//!     client.partition_client(
//!         "my_topic",
//!         0,
//!         UnknownTopicHandling::Retry,
//!     ).await.unwrap()
//! );
//!
//! // construct stream consumer
//! let mut stream = StreamConsumerBuilder::new(
//!         partition_client,
//!         StartOffset::Earliest,
//!     )
//!     .with_max_wait_ms(100)
//!     .build();
//!
//! // consume data
//! let (record, high_water_mark) = stream
//!     .next()
//!     .await
//!     .expect("some records")
//!     .expect("no error");
//! # }
//! ```
use std::collections::VecDeque;
use std::ops::Range;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::time::Duration;

use futures::Stream;
use futures::future::{BoxFuture, Fuse, FusedFuture, FutureExt};
use tracing::{debug, trace, warn};

use crate::{
    client::{
        error::{Error, ProtocolError, Result},
        partition::PartitionClient,
    },
    record::RecordAndOffset,
};

use super::partition::OffsetAt;

/// At which position shall the stream start.
#[derive(Debug, Clone, Copy)]
pub enum StartOffset {
    /// At the earlist known offset.
    ///
    /// This might be larger than 0 if some records were already deleted due to a retention policy.
    Earliest,

    /// At the latest known offset.
    ///
    /// This is helpful if you only want ot process new data.
    Latest,

    /// At a specific offset.
    ///
    /// Note that specifying an offset that is unknown to the broker will result in a [`Error::ServerError`] with
    /// [`ProtocolError::OffsetOutOfRange`] and the stream will terminate right after the error.
    At(i64),
}

#[derive(Debug)]
pub struct StreamConsumerBuilder {
    client: Arc<dyn FetchClient>,

    start_offset: StartOffset,

    max_wait_ms: i32,

    min_batch_size: i32,

    max_batch_size: i32,
}

impl StreamConsumerBuilder {
    pub fn new(client: Arc<PartitionClient>, start_offset: StartOffset) -> Self {
        Self::new_with_client(client, start_offset)
    }

    /// Internal API for creating with any `dyn FetchClient`
    fn new_with_client(client: Arc<dyn FetchClient>, start_offset: StartOffset) -> Self {
        Self {
            client,
            start_offset,
            // Use same defaults as rdkafka:
            // - <https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md>
            max_wait_ms: 500,
            min_batch_size: 1,
            max_batch_size: 52428800,
        }
    }

    /// Will wait for at least `min_batch_size` bytes of data
    pub fn with_min_batch_size(self, min_batch_size: i32) -> Self {
        Self {
            min_batch_size,
            ..self
        }
    }

    /// The maximum amount of data to fetch in a single batch
    pub fn with_max_batch_size(self, max_batch_size: i32) -> Self {
        Self {
            max_batch_size,
            ..self
        }
    }

    /// The maximum amount of time to wait for data before returning
    pub fn with_max_wait_ms(self, max_wait_ms: i32) -> Self {
        Self {
            max_wait_ms,
            ..self
        }
    }

    pub fn build(self) -> StreamConsumer {
        StreamConsumer {
            client: self.client,
            max_wait_ms: self.max_wait_ms,
            min_batch_size: self.min_batch_size,
            max_batch_size: self.max_batch_size,
            next_offset: None,
            next_backoff: None,
            start_offset: self.start_offset,
            terminated: false,
            last_high_watermark: -1,
            buffer: Default::default(),
            fetch_fut: Fuse::terminated(),
        }
    }
}

struct FetchResultOk {
    records_and_offsets: Vec<RecordAndOffset>,
    watermark: i64,
    used_offset: i64,
}

type FetchResult = Result<FetchResultOk>;

/// A trait wrapper to allow mocking
trait FetchClient: std::fmt::Debug + Send + Sync {
    /// Fetch records.
    ///
    /// Arguments are identical to [`PartitionClient::fetch_records`].
    fn fetch_records(
        &self,
        offset: i64,
        bytes: Range<i32>,
        max_wait_ms: i32,
    ) -> BoxFuture<'_, Result<(Vec<RecordAndOffset>, i64)>>;

    /// Get offset.
    ///
    /// Arguments are identical to [`PartitionClient::get_offset`].
    fn get_offset(&self, at: OffsetAt) -> BoxFuture<'_, Result<i64>>;
}

impl FetchClient for PartitionClient {
    fn fetch_records(
        &self,
        offset: i64,
        bytes: Range<i32>,
        max_wait_ms: i32,
    ) -> BoxFuture<'_, Result<(Vec<RecordAndOffset>, i64)>> {
        Box::pin(self.fetch_records(offset, bytes, max_wait_ms))
    }

    fn get_offset(&self, at: OffsetAt) -> BoxFuture<'_, Result<i64>> {
        Box::pin(self.get_offset(at))
    }
}

/// Stream consuming data from start offset.
///
/// # Error Handling
/// If an error is returned by [`fetch_records`](`PartitionClient::fetch_records`) then the stream will emit this error
/// once and will terminate afterwards.
pub struct StreamConsumer {
    client: Arc<dyn FetchClient>,

    min_batch_size: i32,

    max_batch_size: i32,

    max_wait_ms: i32,

    start_offset: StartOffset,

    next_offset: Option<i64>,

    next_backoff: Option<Duration>,

    terminated: bool,

    last_high_watermark: i64,

    buffer: VecDeque<RecordAndOffset>,

    fetch_fut: Fuse<BoxFuture<'static, FetchResult>>,
}

impl Stream for StreamConsumer {
    type Item = Result<(RecordAndOffset, i64)>;

    fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        loop {
            if self.terminated {
                return Poll::Ready(None);
            }
            if let Some(x) = self.buffer.pop_front() {
                return Poll::Ready(Some(Ok((x, self.last_high_watermark))));
            }

            if self.fetch_fut.is_terminated() {
                let next_offset = self.next_offset;
                let start_offset = self.start_offset;
                let bytes = (self.min_batch_size)..(self.max_batch_size);
                let max_wait_ms = self.max_wait_ms;
                let next_backoff = std::mem::take(&mut self.next_backoff);
                let client = Arc::clone(&self.client);

                trace!(?start_offset, ?next_offset, "Fetching records at offset");

                self.fetch_fut = FutureExt::fuse(Box::pin(async move {
                    if let Some(backoff) = next_backoff {
                        tokio::time::sleep(backoff).await;
                    }

                    let offset = match next_offset {
                        Some(x) => x,
                        None => match start_offset {
                            StartOffset::Earliest => {
                                let offset = client.get_offset(OffsetAt::Earliest).await?;
                                debug!(offset, "resolved `earliest` offset");
                                offset
                            }
                            StartOffset::Latest => {
                                let offset = client.get_offset(OffsetAt::Latest).await?;
                                debug!(offset, "resolved `latest` offset");
                                offset
                            }
                            StartOffset::At(x) => x,
                        },
                    };

                    let (records_and_offsets, watermark) =
                        client.fetch_records(offset, bytes, max_wait_ms).await?;
                    Ok(FetchResultOk {
                        records_and_offsets,
                        watermark,
                        used_offset: offset,
                    })
                }));
            }

            let data: FetchResult = futures::ready!(self.fetch_fut.poll_unpin(cx));

            match (data, self.start_offset) {
                (Ok(inner), _) => {
                    let FetchResultOk {
                        mut records_and_offsets,
                        watermark,
                        used_offset,
                    } = inner;
                    trace!(
                        high_watermark = watermark,
                        n_records = records_and_offsets.len(),
                        "Received records and a high watermark",
                    );

                    // Remember used offset (might be overwritten if there was any data) so we don't refetch the
                    // earliest / latest offset for every try. Also fetching the latest offset might be racy otherwise,
                    // since we'll never be in a position where the latest one can actually be fetched.
                    self.next_offset = Some(used_offset);

                    // Sort records by offset in case they aren't in order
                    records_and_offsets.sort_by_key(|x| x.offset);
                    self.last_high_watermark = watermark;
                    if let Some(x) = records_and_offsets.last() {
                        self.next_offset = Some(x.offset + 1);
                        self.buffer.extend(records_and_offsets)
                    }
                    continue;
                }
                // if we don't have an offset, try again because fetching the offset is racy
                (
                    Err(Error::ServerError {
                        protocol_error: ProtocolError::OffsetOutOfRange,
                        ..
                    }),
                    StartOffset::Earliest | StartOffset::Latest,
                ) => {
                    // wipe offset and try again
                    self.next_offset = None;

                    // This will only happen if retention / deletions happen after we've asked for the earliest/latest
                    // offset and our "fetch" request. This should be a rather rare event, but if something is horrible
                    // wrong in our cluster (e.g. some actor is spamming "delete" requests) then let's at least backoff
                    // a bit.
                    let backoff_secs = 1;
                    warn!(
                        start_offset=?self.start_offset,
                        backoff_secs,
                        "Records are gone between ListOffsets and Fetch, backoff a bit",
                    );
                    self.next_backoff = Some(Duration::from_secs(backoff_secs));

                    continue;
                }
                // if we have an offset, terminate the stream
                (Err(e), _) => {
                    self.terminated = true;

                    // report error once
                    return Poll::Ready(Some(Err(e)));
                }
            }
        }
    }
}

impl std::fmt::Debug for StreamConsumer {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("StreamConsumer")
            .field("client", &self.client)
            .field("min_batch_size", &self.min_batch_size)
            .field("max_batch_size", &self.max_batch_size)
            .field("max_wait_ms", &self.max_wait_ms)
            .field("next_offset", &self.next_offset)
            .field("terminated", &self.terminated)
            .field("last_high_watermark", &self.last_high_watermark)
            .field("buffer", &self.buffer)
            .finish_non_exhaustive()
    }
}

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

    use assert_matches::assert_matches;
    use chrono::{TimeZone, Utc};
    use futures::{StreamExt, pin_mut};
    use tokio::sync::{Mutex, mpsc};

    use crate::{
        client::error::{Error, ProtocolError, RequestContext},
        record::Record,
    };

    use super::*;

    #[derive(Debug)]
    struct MockFetch {
        inner: Arc<Mutex<MockFetchInner>>,
    }

    #[derive(Debug)]
    struct MockFetchInner {
        batch_sizes: Vec<usize>,
        stream: mpsc::Receiver<Record>,
        next_err: Option<Error>,
        buffer: Vec<Record>,
        range: (i64, i64),
    }

    impl MockFetch {
        fn new(stream: mpsc::Receiver<Record>, next_err: Option<Error>, range: (i64, i64)) -> Self {
            Self {
                inner: Arc::new(Mutex::new(MockFetchInner {
                    batch_sizes: vec![],
                    stream,
                    buffer: Default::default(),
                    next_err,
                    range,
                })),
            }
        }

        async fn batch_sizes(&self) -> Vec<usize> {
            self.inner.lock().await.batch_sizes.clone()
        }
    }

    impl FetchClient for MockFetch {
        fn fetch_records(
            &self,
            start_offset: i64,
            bytes: Range<i32>,
            max_wait_ms: i32,
        ) -> BoxFuture<'_, Result<(Vec<RecordAndOffset>, i64)>> {
            let inner = Arc::clone(&self.inner);
            Box::pin(async move {
                if let Some(err) = inner.lock().await.next_err.take() {
                    return Err(err);
                }

                println!("MockFetch::fetch_records");
                let mut inner = inner.lock().await;
                println!("MockFetch::fetch_records locked");

                let mut buffer = vec![];
                let mut buffered = 0;

                // Drain input queue
                while let Ok(x) = inner.stream.try_recv() {
                    inner.buffer.push(x)
                }

                for (record_offset, record) in
                    inner.buffer.iter().enumerate().skip(start_offset as usize)
                {
                    let size = record.approximate_size();
                    if size + buffered > bytes.end as usize {
                        assert_ne!(buffered, 0, "record too large");
                        break;
                    }

                    buffer.push(RecordAndOffset {
                        record: record.clone(),
                        offset: record_offset as i64,
                    });
                    buffered += size;
                }

                println!("Waiting up to {} ms for more data", max_wait_ms);

                // Need to wait for more data
                let timeout = tokio::time::sleep(Duration::from_millis(max_wait_ms as u64)).fuse();
                pin_mut!(timeout);

                while buffered < bytes.start as usize && !timeout.is_terminated() {
                    futures::select! {
                        maybe_record = inner.stream.recv().fuse() => match maybe_record {
                            Some(record) => {
                                println!("Received a new record");
                                let size = record.approximate_size();
                                let record_offset = inner.buffer.len() as i64;

                                // Remember record for later
                                inner.buffer.push(record.clone());

                                if record_offset < start_offset {
                                    continue
                                }

                                if size + buffered > bytes.end as usize {
                                    assert_ne!(buffered, 0, "record too large");
                                    break;
                                }

                                buffer.push(RecordAndOffset {
                                    record,
                                    offset: record_offset,
                                });
                                buffered += size
                            }
                            None => break,
                        },
                        _ = timeout => {
                            println!("Timeout receiving records");
                            break
                        },
                    }
                }

                inner.batch_sizes.push(buffer.len());

                Ok((buffer, inner.buffer.len() as i64 - 1))
            })
        }

        fn get_offset(&self, at: OffsetAt) -> BoxFuture<'_, Result<i64>> {
            let inner = Arc::clone(&self.inner);

            Box::pin(async move {
                match at {
                    OffsetAt::Earliest => Ok(inner.lock().await.range.0),
                    OffsetAt::Latest => Ok(inner.lock().await.range.1),
                    OffsetAt::Timestamp(_) => {
                        unreachable!("timestamp based offset is tested in e2e test")
                    }
                }
            })
        }
    }

    #[tokio::test]
    async fn test_consumer() {
        let record = Record {
            key: Some(vec![0; 4]),
            value: Some(vec![0; 6]),
            headers: Default::default(),
            timestamp: Utc.timestamp_millis_opt(1337).unwrap(),
        };

        let (sender, receiver) = mpsc::channel(10);
        let consumer = Arc::new(MockFetch::new(receiver, None, (0, 1_000)));
        let mut stream = StreamConsumerBuilder::new_with_client(
            Arc::<MockFetch>::clone(&consumer),
            StartOffset::At(2),
        )
        .with_max_wait_ms(10)
        .build();

        assert_stream_pending(&mut stream).await;

        // Write two records, nothing should happen as start offset is 2
        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();

        assert_stream_pending(&mut stream).await;

        sender.send(record.clone()).await.unwrap();

        let unwrap = |e: Result<Option<Result<_, _>>, _>| e.unwrap().unwrap().unwrap();

        let (record_and_offset, high_watermark) =
            unwrap(tokio::time::timeout(Duration::from_micros(10), stream.next()).await);

        assert_eq!(record_and_offset.offset, 2);
        assert_eq!(high_watermark, 2);

        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();

        let (record_and_offset, high_watermark) =
            unwrap(tokio::time::timeout(Duration::from_micros(1), stream.next()).await);
        assert_eq!(record_and_offset.offset, 3);
        assert_eq!(high_watermark, 5);

        let (record_and_offset, high_watermark) =
            tokio::time::timeout(Duration::from_millis(1), stream.next())
                .await
                .unwrap()
                .unwrap()
                .unwrap();
        assert_eq!(record_and_offset.offset, 4);
        assert_eq!(high_watermark, 5);

        let (record_and_offset, high_watermark) =
            tokio::time::timeout(Duration::from_millis(1), stream.next())
                .await
                .unwrap()
                .unwrap()
                .unwrap();
        assert_eq!(record_and_offset.offset, 5);
        assert_eq!(high_watermark, 5);

        let received = consumer.batch_sizes().await;
        assert_eq!(&received, &[1, 3]);
    }

    #[tokio::test]
    async fn test_consumer_timeout() {
        let record = Record {
            key: Some(vec![0; 4]),
            value: Some(vec![0; 6]),
            headers: Default::default(),
            timestamp: Utc.timestamp_millis_opt(1337).unwrap(),
        };

        let (sender, receiver) = mpsc::channel(10);
        let consumer = Arc::new(MockFetch::new(receiver, None, (0, 1_000)));

        assert!(consumer.batch_sizes().await.is_empty());

        let mut stream = StreamConsumerBuilder::new_with_client(
            Arc::<MockFetch>::clone(&consumer),
            StartOffset::At(0),
        )
        .with_min_batch_size((record.approximate_size() * 2) as i32)
        .with_max_batch_size((record.approximate_size() * 3) as i32)
        .with_max_wait_ms(5)
        .build();

        // Should return nothing
        assert_stream_pending(&mut stream).await;

        // Stream might be holding lock, so must poll it whilst trying to extract batch sizes
        // to allow timeouts to be serviced and the lock released
        let received = tokio::select! {
            _ = stream.next() => panic!("stream returned!"),
            x = consumer.batch_sizes() => x,
        };

        // Should have had some requests timeout returning no records
        assert!(!received.is_empty());
        assert!(received.iter().all(|x| *x == 0));

        sender.send(record.clone()).await.unwrap();

        // Should wait for max_wait_ms
        tokio::time::timeout(Duration::from_millis(10), stream.next())
            .await
            .unwrap()
            .unwrap()
            .unwrap();

        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();

        // Should not wait for max_wait_ms
        tokio::time::timeout(Duration::from_micros(1), stream.next())
            .await
            .unwrap()
            .unwrap()
            .unwrap();
        // Should not wait for max_wait_ms
        tokio::time::timeout(Duration::from_micros(1), stream.next())
            .await
            .unwrap()
            .unwrap()
            .unwrap();
    }

    #[tokio::test]
    async fn test_consumer_terminate() {
        let e = Error::ServerError {
            protocol_error: ProtocolError::OffsetOutOfRange,
            error_message: None,
            request: RequestContext::Partition("foo".into(), 1),
            response: None,
            is_virtual: true,
        };
        let (_sender, receiver) = mpsc::channel(10);
        let consumer = Arc::new(MockFetch::new(receiver, Some(e), (0, 1_000)));

        let mut stream =
            StreamConsumerBuilder::new_with_client(consumer, StartOffset::At(0)).build();

        let error = stream.next().await.expect("stream not empty").unwrap_err();
        assert_matches!(
            error,
            Error::ServerError {
                protocol_error: ProtocolError::OffsetOutOfRange,
                ..
            }
        );

        // stream ends
        assert!(stream.next().await.is_none());
    }

    #[tokio::test]
    async fn test_consumer_earliest() {
        let record = Record {
            key: Some(vec![0; 4]),
            value: Some(vec![0; 6]),
            headers: Default::default(),
            timestamp: Utc.timestamp_millis_opt(1337).unwrap(),
        };

        // Simulate an error on first fetch to encourage an offset update
        let e = Error::ServerError {
            protocol_error: ProtocolError::OffsetOutOfRange,
            error_message: None,
            request: RequestContext::Partition("foo".into(), 1),
            response: None,
            is_virtual: true,
        };

        let (sender, receiver) = mpsc::channel(10);
        let consumer = Arc::new(MockFetch::new(receiver, Some(e), (2, 1_000)));
        let mut stream = StreamConsumerBuilder::new_with_client(
            Arc::<MockFetch>::clone(&consumer),
            StartOffset::Earliest,
        )
        .with_max_wait_ms(10)
        .build();

        assert_stream_pending(&mut stream).await;

        // Write two records, nothing should happen as start offset is 2 (via "earliest")
        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();

        assert_stream_pending(&mut stream).await;

        sender.send(record.clone()).await.unwrap();

        let unwrap = |e: Result<Option<Result<_, _>>, _>| e.unwrap().unwrap().unwrap();

        // need a solid timeout here because we have simulated an error that caused a backoff
        let (record_and_offset, high_watermark) =
            unwrap(tokio::time::timeout(Duration::from_secs(2), stream.next()).await);

        assert_eq!(record_and_offset.offset, 2);
        assert_eq!(high_watermark, 2);
    }

    #[tokio::test]
    async fn test_consumer_latest() {
        let record = Record {
            key: Some(vec![0; 4]),
            value: Some(vec![0; 6]),
            headers: Default::default(),
            timestamp: Utc.timestamp_millis_opt(1337).unwrap(),
        };

        // Simulate an error on first fetch to encourage an offset update
        let e = Error::ServerError {
            protocol_error: ProtocolError::OffsetOutOfRange,
            error_message: None,
            request: RequestContext::Partition("foo".into(), 1),
            response: None,
            is_virtual: true,
        };

        let (sender, receiver) = mpsc::channel(10);
        let consumer = Arc::new(MockFetch::new(receiver, Some(e), (0, 2)));
        let mut stream = StreamConsumerBuilder::new_with_client(
            Arc::<MockFetch>::clone(&consumer),
            StartOffset::Latest,
        )
        .with_max_wait_ms(10)
        .build();

        assert_stream_pending(&mut stream).await;

        // Write two records, nothing should happen as start offset is 2 (via "latest")
        sender.send(record.clone()).await.unwrap();
        sender.send(record.clone()).await.unwrap();

        assert_stream_pending(&mut stream).await;

        sender.send(record.clone()).await.unwrap();

        let unwrap = |e: Result<Option<Result<_, _>>, _>| e.unwrap().unwrap().unwrap();

        // need a solid timeout here because we have simulated an error that caused a backoff
        let (record_and_offset, high_watermark) =
            unwrap(tokio::time::timeout(Duration::from_secs(2), stream.next()).await);

        assert_eq!(record_and_offset.offset, 2);
        assert_eq!(high_watermark, 2);
    }

    /// Assert that given stream is pending.
    ///
    /// This will will try to poll the stream for a bit to ensure that async IO has a chance to catch up.
    async fn assert_stream_pending<S>(stream: &mut S)
    where
        S: Stream + Send + Unpin,
        S::Item: std::fmt::Debug,
    {
        tokio::select! {
            e = stream.next() => panic!("stream is not pending, yielded: {e:?}"),
            _ = tokio::time::sleep(Duration::from_millis(1)) => {},
        };
    }
}