@@ -495,7 +495,7 @@ func makeTestSequence(n int) []Message {
495
495
}
496
496
497
497
func prepareReader (t * testing.T , ctx context.Context , r * Reader , msgs ... Message ) {
498
- var config = r .Config ()
498
+ config : = r .Config ()
499
499
var conn * Conn
500
500
var err error
501
501
@@ -710,7 +710,6 @@ func TestReaderPartitionWhenConsumerGroupsEnabled(t *testing.T) {
710
710
if ! invoke () {
711
711
t .Fatalf ("expected panic; but NewReader worked?!" )
712
712
}
713
-
714
713
}
715
714
716
715
func TestExtractTopics (t * testing.T ) {
@@ -1281,6 +1280,53 @@ func TestValidateReader(t *testing.T) {
1281
1280
}
1282
1281
}
1283
1282
1283
+ func TestCommitLoopImmediateFlushOnGenerationEnd (t * testing.T ) {
1284
+ t .Parallel ()
1285
+ var committedOffset int64
1286
+ var commitCount int
1287
+ gen := & Generation {
1288
+ conn : mockCoordinator {
1289
+ offsetCommitFunc : func (r offsetCommitRequestV2 ) (offsetCommitResponseV2 , error ) {
1290
+ commitCount ++
1291
+ committedOffset = r .Topics [0 ].Partitions [0 ].Offset
1292
+ return offsetCommitResponseV2 {}, nil
1293
+ },
1294
+ },
1295
+ done : make (chan struct {}),
1296
+ log : func (func (Logger )) {},
1297
+ logError : func (func (Logger )) {},
1298
+ }
1299
+
1300
+ // initialize commits so that the commitLoopImmediate select statement blocks
1301
+ r := & Reader {stctx : context .Background (), commits : make (chan commitRequest , 100 )}
1302
+
1303
+ for i := 0 ; i < 100 ; i ++ {
1304
+ cr := commitRequest {
1305
+ commits : []commit {{
1306
+ topic : "topic" ,
1307
+ partition : 0 ,
1308
+ offset : int64 (i ) + 1 ,
1309
+ }},
1310
+ errch : make (chan <- error , 1 ),
1311
+ }
1312
+ r .commits <- cr
1313
+ }
1314
+
1315
+ gen .Start (func (ctx context.Context ) {
1316
+ r .commitLoopImmediate (ctx , gen )
1317
+ })
1318
+
1319
+ gen .close ()
1320
+
1321
+ if committedOffset != 100 {
1322
+ t .Fatalf ("expected commited offset to be 100 but got %d" , committedOffset )
1323
+ }
1324
+
1325
+ if commitCount >= 100 {
1326
+ t .Fatalf ("expected a single final commit on generation end got %d" , commitCount )
1327
+ }
1328
+ }
1329
+
1284
1330
func TestCommitOffsetsWithRetry (t * testing.T ) {
1285
1331
offsets := offsetStash {"topic" : {0 : 0 }}
1286
1332
0 commit comments