Skip to content

Commit fe6006a

Browse files
committed
fix: backoff a bit when earliest/latest offset moves
1 parent 3d0f7d6 commit fe6006a

File tree

1 file changed

+28
-4
lines changed

1 file changed

+28
-4
lines changed

src/client/consumer.rs

Lines changed: 28 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -41,11 +41,12 @@ use std::ops::Range;
4141
use std::pin::Pin;
4242
use std::sync::Arc;
4343
use std::task::{Context, Poll};
44+
use std::time::Duration;
4445

4546
use futures::future::{BoxFuture, Fuse, FusedFuture, FutureExt};
4647
use futures::Stream;
4748
use pin_project_lite::pin_project;
48-
use tracing::trace;
49+
use tracing::{trace, warn};
4950

5051
use crate::{
5152
client::{
@@ -139,6 +140,7 @@ impl StreamConsumerBuilder {
139140
min_batch_size: self.min_batch_size,
140141
max_batch_size: self.max_batch_size,
141142
next_offset: None,
143+
next_backoff: None,
142144
start_offset: self.start_offset,
143145
terminated: false,
144146
last_high_watermark: -1,
@@ -208,6 +210,8 @@ pin_project! {
208210

209211
next_offset: Option<i64>,
210212

213+
next_backoff: Option<Duration>,
214+
211215
terminated: bool,
212216

213217
last_high_watermark: i64,
@@ -236,16 +240,21 @@ impl Stream for StreamConsumer {
236240
let start_offset = *this.start_offset;
237241
let bytes = (*this.min_batch_size)..(*this.max_batch_size);
238242
let max_wait_ms = *this.max_wait_ms;
243+
let next_backoff = std::mem::take(this.next_backoff);
239244
let client = Arc::clone(this.client);
240245

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

243248
*this.fetch_fut = FutureExt::fuse(Box::pin(async move {
249+
if let Some(backoff) = next_backoff {
250+
tokio::time::sleep(backoff).await;
251+
}
252+
244253
let offset = match next_offset {
245254
Some(x) => x,
246255
None => match start_offset {
247256
StartOffset::Earliest => client.get_offset(OffsetAt::Earliest).await?,
248-
StartOffset::Latest => dbg!(client.get_offset(OffsetAt::Latest).await?),
257+
StartOffset::Latest => client.get_offset(OffsetAt::Latest).await?,
249258
StartOffset::At(x) => x,
250259
},
251260
};
@@ -296,6 +305,19 @@ impl Stream for StreamConsumer {
296305
) => {
297306
// wipe offset and try again
298307
*this.next_offset = None;
308+
309+
// This will only happen if retention / deletions happen after we've asked for the earliest/latest
310+
// offset and our "fetch" request. This should be a rather rare event, but if something is horrible
311+
// wrong in our cluster (e.g. some actor is spamming "delete" requests) then let's at least backoff
312+
// a bit.
313+
let backoff_secs = 1;
314+
warn!(
315+
start_offset=?this.start_offset,
316+
backoff_secs,
317+
"Records are gone between ListOffsets and Fetch, backoff a bit",
318+
);
319+
*this.next_backoff = Some(Duration::from_secs(backoff_secs));
320+
299321
continue;
300322
}
301323
// if we have an offset, terminate the stream
@@ -661,8 +683,9 @@ mod tests {
661683

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

686+
// need a solid timeout here because we have simulated an error that caused a backoff
664687
let (record_and_offset, high_watermark) =
665-
unwrap(tokio::time::timeout(Duration::from_micros(10), stream.next()).await);
688+
unwrap(tokio::time::timeout(Duration::from_secs(2), stream.next()).await);
666689

667690
assert_eq!(record_and_offset.offset, 2);
668691
assert_eq!(high_watermark, 2);
@@ -704,8 +727,9 @@ mod tests {
704727

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

730+
// need a solid timeout here because we have simulated an error that caused a backoff
707731
let (record_and_offset, high_watermark) =
708-
unwrap(tokio::time::timeout(Duration::from_micros(10), stream.next()).await);
732+
unwrap(tokio::time::timeout(Duration::from_secs(2), stream.next()).await);
709733

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

0 commit comments

Comments
 (0)