@@ -121,7 +121,7 @@ func (r *Reader) unsubscribe() {
121
121
// another consumer to avoid such a race.
122
122
}
123
123
124
- func (r * Reader ) subscribe (allAssignments map [string ][]PartitionAssignment ) {
124
+ func (r * Reader ) subscribe (generationId int32 , allAssignments map [string ][]PartitionAssignment ) {
125
125
offsets := make (map [topicPartition ]int64 )
126
126
for topic , assignments := range allAssignments {
127
127
for _ , assignment := range assignments {
@@ -134,7 +134,7 @@ func (r *Reader) subscribe(allAssignments map[string][]PartitionAssignment) {
134
134
}
135
135
136
136
r .mutex .Lock ()
137
- r .start (offsets )
137
+ r .start (generationId , offsets )
138
138
r .mutex .Unlock ()
139
139
140
140
r .withLogger (func (l Logger ) {
@@ -150,35 +150,73 @@ func (r *Reader) commitOffsetsWithRetry(gen *Generation, offsetStash offsetStash
150
150
backoffDelayMax = 5 * time .Second
151
151
)
152
152
153
- for attempt := 0 ; attempt < retries ; attempt ++ {
154
- if attempt != 0 {
155
- if ! sleep (r .stctx , backoff (attempt , backoffDelayMin , backoffDelayMax )) {
156
- return
153
+ messagesToSendForGeneration := make (map [int32 ]map [string ]map [int ]int64 )
154
+ for topic , partitionsInfo := range offsetStash {
155
+ for partition , commitInfo := range partitionsInfo {
156
+ if _ , ok := messagesToSendForGeneration [commitInfo .generationID ]; ! ok {
157
+ messagesToSendForGeneration [commitInfo .generationID ] = make (map [string ]map [int ]int64 )
158
+ }
159
+ msgsForTopic := messagesToSendForGeneration [commitInfo .generationID ]
160
+ if _ , ok := msgsForTopic [topic ]; ! ok {
161
+ msgsForTopic [topic ] = make (map [int ]int64 )
157
162
}
163
+ msgsForPartition := msgsForTopic [topic ]
164
+ msgsForPartition [partition ] = commitInfo .offset
158
165
}
166
+ }
167
+ var illegalGenerationErr bool
168
+ for generationID , messages := range messagesToSendForGeneration {
169
+ for attempt := 0 ; attempt < retries ; attempt ++ {
170
+ if attempt != 0 {
171
+ if ! sleep (r .stctx , backoff (attempt , backoffDelayMin , backoffDelayMax )) {
172
+ continue
173
+ }
174
+ }
159
175
160
- if err = gen .CommitOffsets (offsetStash ); err == nil {
161
- return
176
+ if err = gen .CommitOffsetsForGenID (generationID , messages ); err == nil {
177
+ break
178
+ }
179
+
180
+ // IllegalGeneration error is not retriable, but we should attempt to
181
+ // perform the remaining commits
182
+ if errors .Is (err , IllegalGeneration ) {
183
+ r .withErrorLogger (func (l Logger ) { l .Printf ("generation %d - %v" , generationID , err ) })
184
+ offsetStash .removeGenerationID (generationID )
185
+ illegalGenerationErr = true
186
+ err = nil
187
+ break
188
+ }
162
189
}
163
190
}
164
191
192
+ // if configured to ignore the error
193
+ if illegalGenerationErr && r .config .ErrorOnWrongGenerationCommit {
194
+ err = IllegalGeneration
195
+ }
165
196
return // err will not be nil
166
197
}
167
198
168
- // offsetStash holds offsets by topic => partition => offset.
169
- type offsetStash map [string ]map [int ]int64
199
+ // offsetStash holds offsets by topic => partition => offsetEntry.
200
+ type offsetEntry struct {
201
+ offset int64
202
+ generationID int32
203
+ }
204
+ type offsetStash map [string ]map [int ]offsetEntry
170
205
171
206
// merge updates the offsetStash with the offsets from the provided messages.
172
207
func (o offsetStash ) merge (commits []commit ) {
173
208
for _ , c := range commits {
174
209
offsetsByPartition , ok := o [c .topic ]
175
210
if ! ok {
176
- offsetsByPartition = map [int ]int64 {}
211
+ offsetsByPartition = map [int ]offsetEntry {}
177
212
o [c .topic ] = offsetsByPartition
178
213
}
179
214
180
- if offset , ok := offsetsByPartition [c .partition ]; ! ok || c .offset > offset {
181
- offsetsByPartition [c .partition ] = c .offset
215
+ if offset , ok := offsetsByPartition [c .partition ]; ! ok || c .offset > offset .offset {
216
+ offsetsByPartition [c .partition ] = offsetEntry {
217
+ offset : c .offset ,
218
+ generationID : c .generationId ,
219
+ }
182
220
}
183
221
}
184
222
}
@@ -190,6 +228,19 @@ func (o offsetStash) reset() {
190
228
}
191
229
}
192
230
231
+ func (o offsetStash ) removeGenerationID (genID int32 ) {
232
+ for topic , offsetsForTopic := range o {
233
+ for partition , offsetsForPartition := range offsetsForTopic {
234
+ if offsetsForPartition .generationID == genID {
235
+ delete (offsetsForTopic , partition )
236
+ }
237
+ if len (offsetsForTopic ) == 0 {
238
+ delete (o , topic )
239
+ }
240
+ }
241
+ }
242
+ }
243
+
193
244
// commitLoopImmediate handles each commit synchronously.
194
245
func (r * Reader ) commitLoopImmediate (ctx context.Context , gen * Generation ) {
195
246
offsets := offsetStash {}
@@ -329,7 +380,7 @@ func (r *Reader) run(cg *ConsumerGroup) {
329
380
330
381
r .stats .rebalances .observe (1 )
331
382
332
- r .subscribe (gen .Assignments )
383
+ r .subscribe (gen .ID , gen . Assignments )
333
384
334
385
gen .Start (func (ctx context.Context ) {
335
386
r .commitLoop (ctx , gen )
@@ -522,6 +573,10 @@ type ReaderConfig struct {
522
573
// This flag is being added to retain backwards-compatibility, so it will be
523
574
// removed in a future version of kafka-go.
524
575
OffsetOutOfRangeError bool
576
+
577
+ // ErrorOnWrongGenerationCommit indicates that we should return an error when
578
+ // attempting to commit a message to a generation different than the current one.
579
+ ErrorOnWrongGenerationCommit bool
525
580
}
526
581
527
582
// Validate method validates ReaderConfig properties.
@@ -819,7 +874,7 @@ func (r *Reader) FetchMessage(ctx context.Context) (Message, error) {
819
874
r .mutex .Lock ()
820
875
821
876
if ! r .closed && r .version == 0 {
822
- r .start (r .getTopicPartitionOffset ())
877
+ r .start (undefinedGenerationId , r .getTopicPartitionOffset ())
823
878
}
824
879
825
880
version := r .version
@@ -1040,7 +1095,7 @@ func (r *Reader) SetOffset(offset int64) error {
1040
1095
r .offset = offset
1041
1096
1042
1097
if r .version != 0 {
1043
- r .start (r .getTopicPartitionOffset ())
1098
+ r .start (undefinedGenerationId , r .getTopicPartitionOffset ())
1044
1099
}
1045
1100
1046
1101
r .activateReadLag ()
@@ -1178,7 +1233,7 @@ func (r *Reader) readLag(ctx context.Context) {
1178
1233
}
1179
1234
}
1180
1235
1181
- func (r * Reader ) start (offsetsByPartition map [topicPartition ]int64 ) {
1236
+ func (r * Reader ) start (generationId int32 , offsetsByPartition map [topicPartition ]int64 ) {
1182
1237
if r .closed {
1183
1238
// don't start child reader if parent Reader is closed
1184
1239
return
@@ -1216,7 +1271,7 @@ func (r *Reader) start(offsetsByPartition map[topicPartition]int64) {
1216
1271
1217
1272
// backwards-compatibility flags
1218
1273
offsetOutOfRangeError : r .config .OffsetOutOfRangeError ,
1219
- }).run (ctx , offset )
1274
+ }).run (ctx , generationId , offset )
1220
1275
}(ctx , key , offset , & r .join )
1221
1276
}
1222
1277
}
@@ -1253,7 +1308,7 @@ type readerMessage struct {
1253
1308
error error
1254
1309
}
1255
1310
1256
- func (r * reader ) run (ctx context.Context , offset int64 ) {
1311
+ func (r * reader ) run (ctx context.Context , generationId int32 , offset int64 ) {
1257
1312
// This is the reader's main loop, it only ends if the context is canceled
1258
1313
// and will keep attempting to reader messages otherwise.
1259
1314
//
@@ -1306,6 +1361,7 @@ func (r *reader) run(ctx context.Context, offset int64) {
1306
1361
}
1307
1362
continue
1308
1363
}
1364
+ conn .generationId = generationId
1309
1365
1310
1366
// Resetting the attempt counter ensures that if a failure occurs after
1311
1367
// a successful initialization we don't keep increasing the backoff
0 commit comments