@@ -111,6 +111,7 @@ impl StreamConsumerBuilder {
111
111
min_batch_size : self . min_batch_size ,
112
112
max_batch_size : self . max_batch_size ,
113
113
next_offset : self . start_offset ,
114
+ terminated : false ,
114
115
last_high_watermark : -1 ,
115
116
buffer : Default :: default ( ) ,
116
117
fetch_fut : Fuse :: terminated ( ) ,
@@ -122,6 +123,9 @@ type FetchResult = Result<(Vec<RecordAndOffset>, i64)>;
122
123
123
124
/// A trait wrapper to allow mocking
124
125
trait FetchClient : std:: fmt:: Debug + Send + Sync {
126
+ /// Fetch records.
127
+ ///
128
+ /// Arguments are identical to [`PartitionClient::fetch_records`].
125
129
fn fetch_records (
126
130
& self ,
127
131
offset : i64 ,
@@ -142,6 +146,11 @@ impl FetchClient for PartitionClient {
142
146
}
143
147
144
148
pin_project ! {
149
+ /// Stream consuming data from start offset.
150
+ ///
151
+ /// # Error Handling
152
+ /// If an error is returned by [`fetch_records`](`FetchClient::fetch_records`) then the stream will emit this error
153
+ /// once and will terminate afterwards.
145
154
pub struct StreamConsumer {
146
155
client: Arc <dyn FetchClient >,
147
156
@@ -153,6 +162,8 @@ pin_project! {
153
162
154
163
next_offset: i64 ,
155
164
165
+ terminated: bool ,
166
+
156
167
last_high_watermark: i64 ,
157
168
158
169
buffer: VecDeque <RecordAndOffset >,
@@ -167,6 +178,9 @@ impl Stream for StreamConsumer {
167
178
fn poll_next ( self : Pin < & mut Self > , cx : & mut Context < ' _ > ) -> Poll < Option < Self :: Item > > {
168
179
let this = self . project ( ) ;
169
180
loop {
181
+ if * this. terminated {
182
+ return Poll :: Ready ( None ) ;
183
+ }
170
184
if let Some ( x) = this. buffer . pop_front ( ) {
171
185
return Poll :: Ready ( Some ( Ok ( ( x, * this. last_high_watermark ) ) ) ) ;
172
186
}
@@ -203,7 +217,12 @@ impl Stream for StreamConsumer {
203
217
}
204
218
continue ;
205
219
}
206
- Err ( e) => return Poll :: Ready ( Some ( Err ( e) ) ) ,
220
+ Err ( e) => {
221
+ * this. terminated = true ;
222
+
223
+ // report error once
224
+ return Poll :: Ready ( Some ( Err ( e) ) ) ;
225
+ }
207
226
}
208
227
}
209
228
}
@@ -217,21 +236,26 @@ impl std::fmt::Debug for StreamConsumer {
217
236
. field ( "max_batch_size" , & self . max_batch_size )
218
237
. field ( "max_wait_ms" , & self . max_wait_ms )
219
238
. field ( "next_offset" , & self . next_offset )
239
+ . field ( "terminated" , & self . terminated )
220
240
. field ( "last_high_watermark" , & self . last_high_watermark )
221
241
. field ( "buffer" , & self . buffer )
222
- . finish ( )
242
+ . finish_non_exhaustive ( )
223
243
}
224
244
}
225
245
226
246
#[ cfg( test) ]
227
247
mod tests {
228
248
use std:: time:: Duration ;
229
249
250
+ use assert_matches:: assert_matches;
230
251
use futures:: { pin_mut, StreamExt } ;
231
252
use time:: OffsetDateTime ;
232
253
use tokio:: sync:: { mpsc, Mutex } ;
233
254
234
- use crate :: record:: Record ;
255
+ use crate :: {
256
+ client:: error:: { Error , ProtocolError } ,
257
+ record:: Record ,
258
+ } ;
235
259
236
260
use super :: * ;
237
261
@@ -348,6 +372,36 @@ mod tests {
348
372
}
349
373
}
350
374
375
+ #[ derive( Debug ) ]
376
+ struct MockErrFetch {
377
+ inner : Arc < Mutex < Option < Error > > > ,
378
+ }
379
+
380
+ impl MockErrFetch {
381
+ fn new ( e : Error ) -> Self {
382
+ Self {
383
+ inner : Arc :: new ( Mutex :: new ( Some ( e) ) ) ,
384
+ }
385
+ }
386
+ }
387
+
388
+ impl FetchClient for MockErrFetch {
389
+ fn fetch_records (
390
+ & self ,
391
+ _offset : i64 ,
392
+ _bytes : Range < i32 > ,
393
+ _max_wait_ms : i32 ,
394
+ ) -> BoxFuture < ' _ , Result < ( Vec < RecordAndOffset > , i64 ) > > {
395
+ let inner = Arc :: clone ( & self . inner ) ;
396
+ Box :: pin ( async move {
397
+ match inner. lock ( ) . await . take ( ) {
398
+ Some ( e) => Err ( e) ,
399
+ None => panic ! ( "EOF" ) ,
400
+ }
401
+ } )
402
+ }
403
+ }
404
+
351
405
#[ tokio:: test]
352
406
async fn test_consumer ( ) {
353
407
let record = Record {
@@ -471,4 +525,24 @@ mod tests {
471
525
. unwrap ( )
472
526
. unwrap ( ) ;
473
527
}
528
+
529
+ #[ tokio:: test]
530
+ async fn test_consumer_terminate ( ) {
531
+ let e = Error :: ServerError (
532
+ ProtocolError :: OffsetOutOfRange ,
533
+ String :: from ( "offset out of range" ) ,
534
+ ) ;
535
+ let consumer = Arc :: new ( MockErrFetch :: new ( e) ) ;
536
+
537
+ let mut stream = StreamConsumerBuilder :: new_with_client ( consumer, 0 ) . build ( ) ;
538
+
539
+ let error = stream. next ( ) . await . expect ( "stream not empty" ) . unwrap_err ( ) ;
540
+ assert_matches ! (
541
+ error,
542
+ Error :: ServerError ( ProtocolError :: OffsetOutOfRange , _)
543
+ ) ;
544
+
545
+ // stream ends
546
+ assert ! ( stream. next( ) . await . is_none( ) ) ;
547
+ }
474
548
}
0 commit comments