File: transaction_manager.go

package info (click to toggle)
golang-github-ibm-sarama 1.45.1-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 2,964 kB
  • sloc: makefile: 35; sh: 19
file content (930 lines) | stat: -rw-r--r-- 29,934 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
package sarama

import (
	"errors"
	"fmt"
	"strings"
	"sync"
	"time"
)

// ProducerTxnStatusFlag mark current transaction status.
type ProducerTxnStatusFlag int16

const (
	// ProducerTxnFlagUninitialized when txnmgr is created
	ProducerTxnFlagUninitialized ProducerTxnStatusFlag = 1 << iota
	// ProducerTxnFlagInitializing when txnmgr is initializing
	ProducerTxnFlagInitializing
	// ProducerTxnFlagReady when is ready to receive transaction
	ProducerTxnFlagReady
	// ProducerTxnFlagInTransaction when transaction is started
	ProducerTxnFlagInTransaction
	// ProducerTxnFlagEndTransaction when transaction will be committed
	ProducerTxnFlagEndTransaction
	// ProducerTxnFlagInError when having abortable or fatal error
	ProducerTxnFlagInError
	// ProducerTxnFlagCommittingTransaction when committing txn
	ProducerTxnFlagCommittingTransaction
	// ProducerTxnFlagAbortingTransaction when committing txn
	ProducerTxnFlagAbortingTransaction
	// ProducerTxnFlagAbortableError when producer encounter an abortable error
	// Must call AbortTxn in this case.
	ProducerTxnFlagAbortableError
	// ProducerTxnFlagFatalError when producer encounter an fatal error
	// Must Close an recreate it.
	ProducerTxnFlagFatalError
)

func (s ProducerTxnStatusFlag) String() string {
	status := make([]string, 0)
	if s&ProducerTxnFlagUninitialized != 0 {
		status = append(status, "ProducerTxnStateUninitialized")
	}
	if s&ProducerTxnFlagInitializing != 0 {
		status = append(status, "ProducerTxnStateInitializing")
	}
	if s&ProducerTxnFlagReady != 0 {
		status = append(status, "ProducerTxnStateReady")
	}
	if s&ProducerTxnFlagInTransaction != 0 {
		status = append(status, "ProducerTxnStateInTransaction")
	}
	if s&ProducerTxnFlagEndTransaction != 0 {
		status = append(status, "ProducerTxnStateEndTransaction")
	}
	if s&ProducerTxnFlagInError != 0 {
		status = append(status, "ProducerTxnStateInError")
	}
	if s&ProducerTxnFlagCommittingTransaction != 0 {
		status = append(status, "ProducerTxnStateCommittingTransaction")
	}
	if s&ProducerTxnFlagAbortingTransaction != 0 {
		status = append(status, "ProducerTxnStateAbortingTransaction")
	}
	if s&ProducerTxnFlagAbortableError != 0 {
		status = append(status, "ProducerTxnStateAbortableError")
	}
	if s&ProducerTxnFlagFatalError != 0 {
		status = append(status, "ProducerTxnStateFatalError")
	}
	return strings.Join(status, "|")
}

// transactionManager keeps the state necessary to ensure idempotent production
type transactionManager struct {
	producerID         int64
	producerEpoch      int16
	sequenceNumbers    map[string]int32
	mutex              sync.Mutex
	transactionalID    string
	transactionTimeout time.Duration
	client             Client

	// when kafka cluster is at least 2.5.0.
	// used to recover when producer failed.
	coordinatorSupportsBumpingEpoch bool

	// When producer need to bump it's epoch.
	epochBumpRequired bool
	// Record last seen error.
	lastError error

	// Ensure that status is never accessed with a race-condition.
	statusLock sync.RWMutex
	status     ProducerTxnStatusFlag

	// Ensure that only one goroutine will update partitions in current transaction.
	partitionInTxnLock            sync.Mutex
	pendingPartitionsInCurrentTxn topicPartitionSet
	partitionsInCurrentTxn        topicPartitionSet

	// Offsets to add to transaction.
	offsetsInCurrentTxn map[string]topicPartitionOffsets
}

const (
	noProducerID    = -1
	noProducerEpoch = -1

	// see publishTxnPartitions comment.
	addPartitionsRetryBackoff = 20 * time.Millisecond
)

// txnmngr allowed transitions.
var producerTxnTransitions = map[ProducerTxnStatusFlag][]ProducerTxnStatusFlag{
	ProducerTxnFlagUninitialized: {
		ProducerTxnFlagReady,
		ProducerTxnFlagInError,
	},
	// When we need are initializing
	ProducerTxnFlagInitializing: {
		ProducerTxnFlagInitializing,
		ProducerTxnFlagReady,
		ProducerTxnFlagInError,
	},
	// When we have initialized transactional producer
	ProducerTxnFlagReady: {
		ProducerTxnFlagInTransaction,
	},
	// When beginTxn has been called
	ProducerTxnFlagInTransaction: {
		// When calling commit or abort
		ProducerTxnFlagEndTransaction,
		// When got an error
		ProducerTxnFlagInError,
	},
	ProducerTxnFlagEndTransaction: {
		// When epoch bump
		ProducerTxnFlagInitializing,
		// When commit is good
		ProducerTxnFlagReady,
		// When got an error
		ProducerTxnFlagInError,
	},
	// Need to abort transaction
	ProducerTxnFlagAbortableError: {
		// Call AbortTxn
		ProducerTxnFlagAbortingTransaction,
		// When got an error
		ProducerTxnFlagInError,
	},
	// Need to close producer
	ProducerTxnFlagFatalError: {
		ProducerTxnFlagFatalError,
	},
}

type topicPartition struct {
	topic     string
	partition int32
}

// to ensure that we don't do a full scan every time a partition or an offset is added.
type (
	topicPartitionSet     map[topicPartition]struct{}
	topicPartitionOffsets map[topicPartition]*PartitionOffsetMetadata
)

func (s topicPartitionSet) mapToRequest() map[string][]int32 {
	result := make(map[string][]int32, len(s))
	for tp := range s {
		result[tp.topic] = append(result[tp.topic], tp.partition)
	}
	return result
}

func (s topicPartitionOffsets) mapToRequest() map[string][]*PartitionOffsetMetadata {
	result := make(map[string][]*PartitionOffsetMetadata, len(s))
	for tp, offset := range s {
		result[tp.topic] = append(result[tp.topic], offset)
	}
	return result
}

// Return true if current transition is allowed.
func (t *transactionManager) isTransitionValid(target ProducerTxnStatusFlag) bool {
	for status, allowedTransitions := range producerTxnTransitions {
		if status&t.status != 0 {
			for _, allowedTransition := range allowedTransitions {
				if allowedTransition&target != 0 {
					return true
				}
			}
		}
	}
	return false
}

// Get current transaction status.
func (t *transactionManager) currentTxnStatus() ProducerTxnStatusFlag {
	t.statusLock.RLock()
	defer t.statusLock.RUnlock()

	return t.status
}

// Try to transition to a valid status and return an error otherwise.
func (t *transactionManager) transitionTo(target ProducerTxnStatusFlag, err error) error {
	t.statusLock.Lock()
	defer t.statusLock.Unlock()

	if !t.isTransitionValid(target) {
		return ErrTransitionNotAllowed
	}

	if target&ProducerTxnFlagInError != 0 {
		if err == nil {
			return ErrCannotTransitionNilError
		}
		t.lastError = err
	} else {
		t.lastError = nil
	}

	DebugLogger.Printf("txnmgr/transition [%s] transition from %s to %s\n", t.transactionalID, t.status, target)

	t.status = target
	return err
}

func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) (int32, int16) {
	key := fmt.Sprintf("%s-%d", topic, partition)
	t.mutex.Lock()
	defer t.mutex.Unlock()
	sequence := t.sequenceNumbers[key]
	t.sequenceNumbers[key] = sequence + 1
	return sequence, t.producerEpoch
}

func (t *transactionManager) bumpEpoch() {
	t.mutex.Lock()
	defer t.mutex.Unlock()
	t.producerEpoch++
	for k := range t.sequenceNumbers {
		t.sequenceNumbers[k] = 0
	}
}

func (t *transactionManager) getProducerID() (int64, int16) {
	t.mutex.Lock()
	defer t.mutex.Unlock()
	return t.producerID, t.producerEpoch
}

// Compute retry backoff considered current attempts.
func (t *transactionManager) computeBackoff(attemptsRemaining int) time.Duration {
	if t.client.Config().Producer.Transaction.Retry.BackoffFunc != nil {
		maxRetries := t.client.Config().Producer.Transaction.Retry.Max
		retries := maxRetries - attemptsRemaining
		return t.client.Config().Producer.Transaction.Retry.BackoffFunc(retries, maxRetries)
	}
	return t.client.Config().Producer.Transaction.Retry.Backoff
}

// return true is txnmngr is transactinal.
func (t *transactionManager) isTransactional() bool {
	return t.transactionalID != ""
}

// add specified offsets to current transaction.
func (t *transactionManager) addOffsetsToTxn(offsetsToAdd map[string][]*PartitionOffsetMetadata, groupId string) error {
	t.mutex.Lock()
	defer t.mutex.Unlock()

	if t.currentTxnStatus()&ProducerTxnFlagInTransaction == 0 {
		return ErrTransactionNotReady
	}

	if t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 {
		return t.lastError
	}

	if _, ok := t.offsetsInCurrentTxn[groupId]; !ok {
		t.offsetsInCurrentTxn[groupId] = topicPartitionOffsets{}
	}

	for topic, offsets := range offsetsToAdd {
		for _, offset := range offsets {
			tp := topicPartition{topic: topic, partition: offset.Partition}
			t.offsetsInCurrentTxn[groupId][tp] = offset
		}
	}
	return nil
}

// send txnmgnr save offsets to transaction coordinator.
func (t *transactionManager) publishOffsetsToTxn(offsets topicPartitionOffsets, groupId string) (topicPartitionOffsets, error) {
	// First AddOffsetsToTxn
	attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max
	exec := func(run func() (bool, error), err error) error {
		for attemptsRemaining >= 0 {
			var retry bool
			retry, err = run()
			if !retry {
				return err
			}
			backoff := t.computeBackoff(attemptsRemaining)
			Logger.Printf("txnmgr/add-offset-to-txn [%s] retrying after %dms... (%d attempts remaining) (%s)\n",
				t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err)
			time.Sleep(backoff)
			attemptsRemaining--
		}
		return err
	}
	lastError := exec(func() (bool, error) {
		coordinator, err := t.client.TransactionCoordinator(t.transactionalID)
		if err != nil {
			return true, err
		}
		request := &AddOffsetsToTxnRequest{
			TransactionalID: t.transactionalID,
			ProducerEpoch:   t.producerEpoch,
			ProducerID:      t.producerID,
			GroupID:         groupId,
		}
		if t.client.Config().Version.IsAtLeast(V2_7_0_0) {
			// Version 2 adds the support for new error code PRODUCER_FENCED.
			request.Version = 2
		} else if t.client.Config().Version.IsAtLeast(V2_0_0_0) {
			// Version 1 is the same as version 0.
			request.Version = 1
		}
		response, err := coordinator.AddOffsetsToTxn(request)
		if err != nil {
			// If an error occurred try to refresh current transaction coordinator.
			_ = coordinator.Close()
			_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			return true, err
		}
		if response == nil {
			// If no response is returned just retry.
			return true, ErrTxnUnableToParseResponse
		}
		if response.Err == ErrNoError {
			DebugLogger.Printf("txnmgr/add-offset-to-txn [%s] successful add-offset-to-txn with group %s %+v\n",
				t.transactionalID, groupId, response)
			// If no error, just exit.
			return false, nil
		}
		switch response.Err {
		case ErrConsumerCoordinatorNotAvailable:
			fallthrough
		case ErrNotCoordinatorForConsumer:
			_ = coordinator.Close()
			_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			fallthrough
		case ErrOffsetsLoadInProgress:
			fallthrough
		case ErrConcurrentTransactions:
			// Retry
		case ErrUnknownProducerID:
			fallthrough
		case ErrInvalidProducerIDMapping:
			return false, t.abortableErrorIfPossible(response.Err)
		case ErrGroupAuthorizationFailed:
			return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, response.Err)
		default:
			// Others are fatal
			return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err)
		}
		return true, response.Err
	}, nil)

	if lastError != nil {
		return offsets, lastError
	}

	resultOffsets := offsets
	// Then TxnOffsetCommit
	// note the result is not completed until the TxnOffsetCommit returns
	attemptsRemaining = t.client.Config().Producer.Transaction.Retry.Max
	execTxnOffsetCommit := func(run func() (topicPartitionOffsets, bool, error), err error) (topicPartitionOffsets, error) {
		var r topicPartitionOffsets
		for attemptsRemaining >= 0 {
			var retry bool
			r, retry, err = run()
			if !retry {
				return r, err
			}
			backoff := t.computeBackoff(attemptsRemaining)
			Logger.Printf("txnmgr/txn-offset-commit [%s] retrying after %dms... (%d attempts remaining) (%s)\n",
				t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err)
			time.Sleep(backoff)
			attemptsRemaining--
		}
		return r, err
	}
	return execTxnOffsetCommit(func() (topicPartitionOffsets, bool, error) {
		consumerGroupCoordinator, err := t.client.Coordinator(groupId)
		if err != nil {
			return resultOffsets, true, err
		}
		request := &TxnOffsetCommitRequest{
			TransactionalID: t.transactionalID,
			ProducerEpoch:   t.producerEpoch,
			ProducerID:      t.producerID,
			GroupID:         groupId,
			Topics:          offsets.mapToRequest(),
		}
		if t.client.Config().Version.IsAtLeast(V2_1_0_0) {
			// Version 2 adds the committed leader epoch.
			request.Version = 2
		} else if t.client.Config().Version.IsAtLeast(V2_0_0_0) {
			// Version 1 is the same as version 0.
			request.Version = 1
		}
		responses, err := consumerGroupCoordinator.TxnOffsetCommit(request)
		if err != nil {
			_ = consumerGroupCoordinator.Close()
			_ = t.client.RefreshCoordinator(groupId)
			return resultOffsets, true, err
		}

		if responses == nil {
			return resultOffsets, true, ErrTxnUnableToParseResponse
		}

		var responseErrors []error
		failedTxn := topicPartitionOffsets{}
		for topic, partitionErrors := range responses.Topics {
			for _, partitionError := range partitionErrors {
				switch partitionError.Err {
				case ErrNoError:
					continue
				// If the topic is unknown or the coordinator is loading, retry with the current coordinator
				case ErrRequestTimedOut:
					fallthrough
				case ErrConsumerCoordinatorNotAvailable:
					fallthrough
				case ErrNotCoordinatorForConsumer:
					_ = consumerGroupCoordinator.Close()
					_ = t.client.RefreshCoordinator(groupId)
					fallthrough
				case ErrUnknownTopicOrPartition:
					fallthrough
				case ErrOffsetsLoadInProgress:
					// Do nothing just retry
				case ErrIllegalGeneration:
					fallthrough
				case ErrUnknownMemberId:
					fallthrough
				case ErrFencedInstancedId:
					fallthrough
				case ErrGroupAuthorizationFailed:
					return resultOffsets, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, partitionError.Err)
				default:
					// Others are fatal
					return resultOffsets, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, partitionError.Err)
				}
				tp := topicPartition{topic: topic, partition: partitionError.Partition}
				failedTxn[tp] = offsets[tp]
				responseErrors = append(responseErrors, partitionError.Err)
			}
		}

		resultOffsets = failedTxn

		if len(resultOffsets) == 0 {
			DebugLogger.Printf("txnmgr/txn-offset-commit [%s] successful txn-offset-commit with group %s\n",
				t.transactionalID, groupId)
			return resultOffsets, false, nil
		}
		return resultOffsets, true, Wrap(ErrTxnOffsetCommit, responseErrors...)
	}, nil)
}

func (t *transactionManager) initProducerId() (int64, int16, error) {
	isEpochBump := false

	req := &InitProducerIDRequest{}
	if t.isTransactional() {
		req.TransactionalID = &t.transactionalID
		req.TransactionTimeout = t.transactionTimeout
	}

	if t.client.Config().Version.IsAtLeast(V2_5_0_0) {
		if t.client.Config().Version.IsAtLeast(V2_7_0_0) {
			// Version 4 adds the support for new error code PRODUCER_FENCED.
			req.Version = 4
		} else {
			// Version 3 adds ProducerId and ProducerEpoch, allowing producers to try
			// to resume after an INVALID_PRODUCER_EPOCH error
			req.Version = 3
		}
		isEpochBump = t.producerID != noProducerID && t.producerEpoch != noProducerEpoch
		t.coordinatorSupportsBumpingEpoch = true
		req.ProducerID = t.producerID
		req.ProducerEpoch = t.producerEpoch
	} else if t.client.Config().Version.IsAtLeast(V2_4_0_0) {
		// Version 2 is the first flexible version.
		req.Version = 2
	} else if t.client.Config().Version.IsAtLeast(V2_0_0_0) {
		// Version 1 is the same as version 0.
		req.Version = 1
	}

	if isEpochBump {
		err := t.transitionTo(ProducerTxnFlagInitializing, nil)
		if err != nil {
			return -1, -1, err
		}
		DebugLogger.Printf("txnmgr/init-producer-id [%s] invoking InitProducerId for the first time in order to acquire a producer ID\n",
			t.transactionalID)
	} else {
		DebugLogger.Printf("txnmgr/init-producer-id [%s] invoking InitProducerId with current producer ID %d and epoch %d in order to bump the epoch\n",
			t.transactionalID, t.producerID, t.producerEpoch)
	}

	attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max
	exec := func(run func() (int64, int16, bool, error), err error) (int64, int16, error) {
		pid := int64(-1)
		pepoch := int16(-1)
		for attemptsRemaining >= 0 {
			var retry bool
			pid, pepoch, retry, err = run()
			if !retry {
				return pid, pepoch, err
			}
			backoff := t.computeBackoff(attemptsRemaining)
			Logger.Printf("txnmgr/init-producer-id [%s] retrying after %dms... (%d attempts remaining) (%s)\n",
				t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err)
			time.Sleep(backoff)
			attemptsRemaining--
		}
		return -1, -1, err
	}
	return exec(func() (int64, int16, bool, error) {
		var err error
		var coordinator *Broker
		if t.isTransactional() {
			coordinator, err = t.client.TransactionCoordinator(t.transactionalID)
		} else {
			coordinator = t.client.LeastLoadedBroker()
		}
		if err != nil {
			return -1, -1, true, err
		}
		response, err := coordinator.InitProducerID(req)
		if err != nil {
			if t.isTransactional() {
				_ = coordinator.Close()
				_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			}
			return -1, -1, true, err
		}
		if response == nil {
			return -1, -1, true, ErrTxnUnableToParseResponse
		}
		if response.Err == ErrNoError {
			if isEpochBump {
				t.sequenceNumbers = make(map[string]int32)
			}
			err := t.transitionTo(ProducerTxnFlagReady, nil)
			if err != nil {
				return -1, -1, true, err
			}
			DebugLogger.Printf("txnmgr/init-producer-id [%s] successful init producer id %+v\n",
				t.transactionalID, response)
			return response.ProducerID, response.ProducerEpoch, false, nil
		}
		switch response.Err {
		// Retriable errors
		case ErrConsumerCoordinatorNotAvailable, ErrNotCoordinatorForConsumer, ErrOffsetsLoadInProgress:
			if t.isTransactional() {
				_ = coordinator.Close()
				_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			}
		// Fatal errors
		default:
			return -1, -1, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err)
		}
		return -1, -1, true, response.Err
	}, nil)
}

// if kafka cluster is at least 2.5.0 mark txnmngr to bump epoch else mark it as fatal.
func (t *transactionManager) abortableErrorIfPossible(err error) error {
	if t.coordinatorSupportsBumpingEpoch {
		t.epochBumpRequired = true
		return t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, err)
	}
	return t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, err)
}

// End current transaction.
func (t *transactionManager) completeTransaction() error {
	if t.epochBumpRequired {
		err := t.transitionTo(ProducerTxnFlagInitializing, nil)
		if err != nil {
			return err
		}
	} else {
		err := t.transitionTo(ProducerTxnFlagReady, nil)
		if err != nil {
			return err
		}
	}

	t.lastError = nil
	t.epochBumpRequired = false
	t.partitionsInCurrentTxn = topicPartitionSet{}
	t.pendingPartitionsInCurrentTxn = topicPartitionSet{}
	t.offsetsInCurrentTxn = map[string]topicPartitionOffsets{}

	return nil
}

// send EndTxn request with commit flag. (true when committing false otherwise)
func (t *transactionManager) endTxn(commit bool) error {
	attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max
	exec := func(run func() (bool, error), err error) error {
		for attemptsRemaining >= 0 {
			var retry bool
			retry, err = run()
			if !retry {
				return err
			}
			backoff := t.computeBackoff(attemptsRemaining)
			Logger.Printf("txnmgr/endtxn [%s] retrying after %dms... (%d attempts remaining) (%s)\n",
				t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err)
			time.Sleep(backoff)
			attemptsRemaining--
		}
		return err
	}
	return exec(func() (bool, error) {
		coordinator, err := t.client.TransactionCoordinator(t.transactionalID)
		if err != nil {
			return true, err
		}
		request := &EndTxnRequest{
			TransactionalID:   t.transactionalID,
			ProducerEpoch:     t.producerEpoch,
			ProducerID:        t.producerID,
			TransactionResult: commit,
		}
		if t.client.Config().Version.IsAtLeast(V2_7_0_0) {
			// Version 2 adds the support for new error code PRODUCER_FENCED.
			request.Version = 2
		} else if t.client.Config().Version.IsAtLeast(V2_0_0_0) {
			// Version 1 is the same as version 0.
			request.Version = 1
		}
		response, err := coordinator.EndTxn(request)
		if err != nil {
			// Always retry on network error
			_ = coordinator.Close()
			_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			return true, err
		}
		if response == nil {
			return true, ErrTxnUnableToParseResponse
		}
		if response.Err == ErrNoError {
			DebugLogger.Printf("txnmgr/endtxn [%s] successful to end txn %+v\n",
				t.transactionalID, response)
			return false, t.completeTransaction()
		}
		switch response.Err {
		// Need to refresh coordinator
		case ErrConsumerCoordinatorNotAvailable:
			fallthrough
		case ErrNotCoordinatorForConsumer:
			_ = coordinator.Close()
			_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			fallthrough
		case ErrOffsetsLoadInProgress:
			fallthrough
		case ErrConcurrentTransactions:
			// Just retry
		case ErrUnknownProducerID:
			fallthrough
		case ErrInvalidProducerIDMapping:
			return false, t.abortableErrorIfPossible(response.Err)
		// Fatal errors
		default:
			return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err)
		}
		return true, response.Err
	}, nil)
}

// We will try to publish associated offsets for each groups
// then send endtxn request to mark transaction as finished.
func (t *transactionManager) finishTransaction(commit bool) error {
	t.mutex.Lock()
	defer t.mutex.Unlock()

	// Ensure no error when committing or aborting
	if commit && t.currentTxnStatus()&ProducerTxnFlagInError != 0 {
		return t.lastError
	} else if !commit && t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 {
		return t.lastError
	}

	// if no records has been sent don't do anything.
	if len(t.partitionsInCurrentTxn) == 0 {
		return t.completeTransaction()
	}

	epochBump := t.epochBumpRequired
	// If we're aborting the transaction, so there should be no need to add offsets.
	if commit && len(t.offsetsInCurrentTxn) > 0 {
		for group, offsets := range t.offsetsInCurrentTxn {
			newOffsets, err := t.publishOffsetsToTxn(offsets, group)
			if err != nil {
				t.offsetsInCurrentTxn[group] = newOffsets
				return err
			}
			delete(t.offsetsInCurrentTxn, group)
		}
	}

	if t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 {
		return t.lastError
	}

	if !errors.Is(t.lastError, ErrInvalidProducerIDMapping) {
		err := t.endTxn(commit)
		if err != nil {
			return err
		}
		if !epochBump {
			return nil
		}
	}
	// reset pid and epoch if needed.
	return t.initializeTransactions()
}

// called before sending any transactional record
// won't do anything if current topic-partition is already added to transaction.
func (t *transactionManager) maybeAddPartitionToCurrentTxn(topic string, partition int32) {
	if t.currentTxnStatus()&ProducerTxnFlagInError != 0 {
		return
	}

	tp := topicPartition{topic: topic, partition: partition}

	t.partitionInTxnLock.Lock()
	defer t.partitionInTxnLock.Unlock()
	if _, ok := t.partitionsInCurrentTxn[tp]; ok {
		// partition is already added
		return
	}

	t.pendingPartitionsInCurrentTxn[tp] = struct{}{}
}

// Makes a request to kafka to add a list of partitions ot the current transaction.
func (t *transactionManager) publishTxnPartitions() error {
	t.partitionInTxnLock.Lock()
	defer t.partitionInTxnLock.Unlock()

	if t.currentTxnStatus()&ProducerTxnFlagInError != 0 {
		return t.lastError
	}

	if len(t.pendingPartitionsInCurrentTxn) == 0 {
		return nil
	}

	// Remove the partitions from the pending set regardless of the result. We use the presence
	// of partitions in the pending set to know when it is not safe to send batches. However, if
	// the partitions failed to be added and we enter an error state, we expect the batches to be
	// aborted anyway. In this case, we must be able to continue sending the batches which are in
	// retry for partitions that were successfully added.
	removeAllPartitionsOnFatalOrAbortedError := func() {
		t.pendingPartitionsInCurrentTxn = topicPartitionSet{}
	}

	// We only want to reduce the backoff when retrying the first AddPartition which errored out due to a
	// CONCURRENT_TRANSACTIONS error since this means that the previous transaction is still completing and
	// we don't want to wait too long before trying to start the new one.
	//
	// This is only a temporary fix, the long term solution is being tracked in
	// https://issues.apache.org/jira/browse/KAFKA-5482
	retryBackoff := t.client.Config().Producer.Transaction.Retry.Backoff
	computeBackoff := func(attemptsRemaining int) time.Duration {
		if t.client.Config().Producer.Transaction.Retry.BackoffFunc != nil {
			maxRetries := t.client.Config().Producer.Transaction.Retry.Max
			retries := maxRetries - attemptsRemaining
			return t.client.Config().Producer.Transaction.Retry.BackoffFunc(retries, maxRetries)
		}
		return retryBackoff
	}
	attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max

	exec := func(run func() (bool, error), err error) error {
		for attemptsRemaining >= 0 {
			var retry bool
			retry, err = run()
			if !retry {
				return err
			}
			backoff := computeBackoff(attemptsRemaining)
			Logger.Printf("txnmgr/add-partition-to-txn retrying after %dms... (%d attempts remaining) (%s)\n", backoff/time.Millisecond, attemptsRemaining, err)
			time.Sleep(backoff)
			attemptsRemaining--
		}
		return err
	}
	return exec(func() (bool, error) {
		coordinator, err := t.client.TransactionCoordinator(t.transactionalID)
		if err != nil {
			return true, err
		}
		request := &AddPartitionsToTxnRequest{
			TransactionalID: t.transactionalID,
			ProducerID:      t.producerID,
			ProducerEpoch:   t.producerEpoch,
			TopicPartitions: t.pendingPartitionsInCurrentTxn.mapToRequest(),
		}
		if t.client.Config().Version.IsAtLeast(V2_7_0_0) {
			// Version 2 adds the support for new error code PRODUCER_FENCED.
			request.Version = 2
		} else if t.client.Config().Version.IsAtLeast(V2_0_0_0) {
			// Version 1 is the same as version 0.
			request.Version = 1
		}
		addPartResponse, err := coordinator.AddPartitionsToTxn(request)
		if err != nil {
			_ = coordinator.Close()
			_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
			return true, err
		}

		if addPartResponse == nil {
			return true, ErrTxnUnableToParseResponse
		}

		// remove from the list partitions that have been successfully updated
		var responseErrors []error
		for topic, results := range addPartResponse.Errors {
			for _, response := range results {
				tp := topicPartition{topic: topic, partition: response.Partition}
				switch response.Err {
				case ErrNoError:
					// Mark partition as added to transaction
					t.partitionsInCurrentTxn[tp] = struct{}{}
					delete(t.pendingPartitionsInCurrentTxn, tp)
					continue
				case ErrConsumerCoordinatorNotAvailable:
					fallthrough
				case ErrNotCoordinatorForConsumer:
					_ = coordinator.Close()
					_ = t.client.RefreshTransactionCoordinator(t.transactionalID)
					fallthrough
				case ErrUnknownTopicOrPartition:
					fallthrough
				case ErrOffsetsLoadInProgress:
					// Retry topicPartition
				case ErrConcurrentTransactions:
					if len(t.partitionsInCurrentTxn) == 0 && retryBackoff > addPartitionsRetryBackoff {
						retryBackoff = addPartitionsRetryBackoff
					}
				case ErrOperationNotAttempted:
					fallthrough
				case ErrTopicAuthorizationFailed:
					removeAllPartitionsOnFatalOrAbortedError()
					return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, response.Err)
				case ErrUnknownProducerID:
					fallthrough
				case ErrInvalidProducerIDMapping:
					removeAllPartitionsOnFatalOrAbortedError()
					return false, t.abortableErrorIfPossible(response.Err)
				// Fatal errors
				default:
					removeAllPartitionsOnFatalOrAbortedError()
					return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err)
				}
				responseErrors = append(responseErrors, response.Err)
			}
		}

		// handle end
		if len(t.pendingPartitionsInCurrentTxn) == 0 {
			DebugLogger.Printf("txnmgr/add-partition-to-txn [%s] successful to add partitions txn %+v\n",
				t.transactionalID, addPartResponse)
			return false, nil
		}
		return true, Wrap(ErrAddPartitionsToTxn, responseErrors...)
	}, nil)
}

// Build a new transaction manager sharing producer client.
func newTransactionManager(conf *Config, client Client) (*transactionManager, error) {
	txnmgr := &transactionManager{
		producerID:                    noProducerID,
		producerEpoch:                 noProducerEpoch,
		client:                        client,
		pendingPartitionsInCurrentTxn: topicPartitionSet{},
		partitionsInCurrentTxn:        topicPartitionSet{},
		offsetsInCurrentTxn:           make(map[string]topicPartitionOffsets),
		status:                        ProducerTxnFlagUninitialized,
	}

	if conf.Producer.Idempotent {
		txnmgr.transactionalID = conf.Producer.Transaction.ID
		txnmgr.transactionTimeout = conf.Producer.Transaction.Timeout
		txnmgr.sequenceNumbers = make(map[string]int32)
		txnmgr.mutex = sync.Mutex{}

		var err error
		txnmgr.producerID, txnmgr.producerEpoch, err = txnmgr.initProducerId()
		if err != nil {
			return nil, err
		}
		Logger.Printf("txnmgr/init-producer-id [%s] obtained a ProducerId: %d and ProducerEpoch: %d\n",
			txnmgr.transactionalID, txnmgr.producerID, txnmgr.producerEpoch)
	}

	return txnmgr, nil
}

// re-init producer-id and producer-epoch if needed.
func (t *transactionManager) initializeTransactions() (err error) {
	t.producerID, t.producerEpoch, err = t.initProducerId()
	return
}