@@ -164,37 +164,9 @@ func (s *GroupTransactSession) End(ctx context.Context, commit TransactionEndTry
164
164
return false , err // we do not abort below, because an error here is ctx closing
165
165
}
166
166
case TryAbort :
167
- // If we have no buffered records, there is no need to abort
168
- // buffered records and we can avoid resetting our producer ID.
169
- if atomic .LoadInt64 (& s .cl .producer .bufferedRecords ) == 0 {
170
- break
171
- }
172
-
173
167
if err := s .cl .AbortBufferedRecords (ctx ); err != nil {
174
168
return false , err // same
175
169
}
176
- defer s .cl .ResetProducerID ()
177
-
178
- allOk := true
179
- s .cl .sinksAndSourcesMu .Lock ()
180
- for _ , sns := range s .cl .sinksAndSources {
181
- allOk = allOk && sns .sink .lastRespSuccessful
182
- }
183
- s .cl .sinksAndSourcesMu .Unlock ()
184
-
185
- if ! allOk {
186
- s .cl .cfg .logger .Log (LogLevelWarn , "Buffered records were aborted, but some sink(s) did not have a final handled produce response. Kafka could still be handling these produce requests or have yet to handle them. We do not want to issue EndTxn before these produce requests are handled, because that would risk beginning a new transaction that we may not finish. Waiting 1s to give Kafka some time... (See KAFKA-12671)" )
187
- timer := time .NewTimer (time .Second )
188
- select {
189
- case <- timer .C :
190
- case <- s .cl .ctx .Done ():
191
- timer .Stop ()
192
- return false , s .cl .ctx .Err ()
193
- case <- ctx .Done ():
194
- timer .Stop ()
195
- return false , ctx .Err ()
196
- }
197
- }
198
170
}
199
171
200
172
wantCommit := bool (commit )
@@ -326,48 +298,51 @@ func (cl *Client) BeginTransaction() error {
326
298
// Records produced during or after a call to this function may not be failed,
327
299
// thus it is incorrect to concurrently produce with this function.
328
300
func (cl * Client ) AbortBufferedRecords (ctx context.Context ) error {
329
- atomic .StoreUint32 (& cl .producer .aborting , 1 )
330
- defer atomic .StoreUint32 (& cl .producer .aborting , 0 )
331
- atomic .AddInt32 (& cl .producer .flushing , 1 ) // disallow lingering to start
332
- defer atomic .AddInt32 (& cl .producer .flushing , - 1 )
301
+ p := & cl .producer
302
+
303
+ atomic .StoreUint32 (& p .aborting , 1 )
304
+ defer atomic .StoreUint32 (& p .aborting , 0 )
305
+ atomic .AddInt32 (& p .flushing , 1 ) // disallow lingering to start
306
+ defer atomic .AddInt32 (& p .flushing , - 1 )
333
307
// At this point, all drain loops that start will immediately stop,
334
308
// thus they will not begin any AddPartitionsToTxn request. We must
335
309
// now wait for any req currently built to be done being issued.
336
310
337
311
cl .cfg .logger .Log (LogLevelInfo , "aborting buffered records" )
338
312
defer cl .cfg .logger .Log (LogLevelDebug , "aborted buffered records" )
339
313
340
- cl .failBufferedRecords (ErrAborting )
314
+ // Similar to flushing, we unlinger; nothing will start a linger because
315
+ // the flushing atomic is non-zero.
316
+ if cl .cfg .linger > 0 || cl .cfg .manualFlushing {
317
+ for _ , parts := range p .topics .load () {
318
+ for _ , part := range parts .load ().partitions {
319
+ part .records .unlingerAndManuallyDrain ()
320
+ }
321
+ }
322
+ }
341
323
342
- // Now, we wait for any active drain to stop. We must wait for all
343
- // workers to stop otherwise we could end up with some exceptionally
344
- // weird scenario where we end a txn and begin a new one before a
345
- // prior AddPartitionsToTxn request that was built is issued.
346
- //
347
- // By waiting for our workers count to hit 0, we know that at that
348
- // point, no new AddPartitionsToTxn request will be sent.
324
+ // We have to wait for all buffered records to either be flushed
325
+ // or to safely abort themselves.
349
326
quit := false
350
327
done := make (chan struct {})
351
328
go func () {
352
- cl . producer .notifyMu .Lock ()
353
- defer cl . producer .notifyMu .Unlock ()
329
+ p .notifyMu .Lock ()
330
+ defer p .notifyMu .Unlock ()
354
331
defer close (done )
355
332
356
- for ! quit && atomic .LoadInt32 ( & cl . producer . workers ) > 0 {
357
- cl . producer .notifyCond .Wait ()
333
+ for ! quit && atomic .LoadInt64 ( & p . bufferedRecords ) > 0 {
334
+ p .notifyCond .Wait ()
358
335
}
359
336
}()
360
337
361
338
select {
362
339
case <- done :
363
- // All records were failed above, and all workers are stopped.
364
- // We are safe to return.
365
340
return nil
366
341
case <- ctx .Done ():
367
- cl . producer .notifyMu .Lock ()
342
+ p .notifyMu .Lock ()
368
343
quit = true
369
- cl . producer .notifyMu .Unlock ()
370
- cl . producer .notifyCond .Broadcast ()
344
+ p .notifyMu .Unlock ()
345
+ p .notifyCond .Broadcast ()
371
346
return ctx .Err ()
372
347
}
373
348
}
0 commit comments