forked from twmb/franz-go
/
producer.go
1088 lines (960 loc) · 31.3 KB
/
producer.go
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
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package kgo
import (
"context"
"errors"
"fmt"
"math"
"sync"
"sync/atomic"
"time"
"github.com/tdx/franz-go/pkg/kerr"
"github.com/tdx/franz-go/pkg/kmsg"
)
type producer struct {
bufferedRecords int64
inflight int64 // high 16: # waiters, low 48: # inflight
cl *Client
topicsMu sync.Mutex // locked to prevent concurrent updates; reads are always atomic
topics *topicsPartitions
// Hooks exist behind a pointer because likely they are not used.
// We only take up one byte vs. 6.
hooks *struct {
buffered []HookProduceRecordBuffered
unbuffered []HookProduceRecordUnbuffered
}
hasHookBatchWritten bool
// unknownTopics buffers all records for topics that are not loaded.
// The map is to a pointer to a slice for reasons documented in
// waitUnknownTopic.
unknownTopicsMu sync.Mutex
unknownTopics map[string]*unknownTopicProduces
id atomic.Value
producingTxn uint32 // 1 if in txn
// We must have a producer field for flushing; we cannot just have a
// field on recBufs that is toggled on flush. If we did, then a new
// recBuf could be created and records sent to while we are flushing.
flushing int32 // >0 if flushing, can Flush many times concurrently
aborting int32 // >0 if aborting, can abort many times concurrently
idMu sync.Mutex
idVersion int16
waitBuffer chan struct{}
// mu and c are used for flush and drain notifications; mu is used for
// a few other tight locks.
mu sync.Mutex
c *sync.Cond
batchPromises ringBatchPromise
promisesMu sync.Mutex
txnMu sync.Mutex
inTxn bool
// If using EndBeginTxnUnsafe, and any partitions are actually produced
// to, we issue an AddPartitionsToTxn at the end to re-add them to a
// new transaction. We have to due to logic races: the broker may not
// have handled the produce requests yet, and we want to ensure a new
// transaction is started.
//
// If the user stops producing, we want to ensure that our restarted
// transaction is actually ended. Thus, we set readded whenever we have
// partitions we actually restart. We issue EndTxn and reset readded in
// EndAndBegin; if nothing more was produced to, we ensure we finish
// the started txn.
readded bool
}
// BufferedProduceRecords returns the number of records currently buffered for
// producing within the client.
//
// This can be used as a gauge to determine how far behind the client is for
// flushing records produced by your client (which can help determine network /
// cluster health).
func (cl *Client) BufferedProduceRecords() int64 {
return atomic.LoadInt64(&cl.producer.bufferedRecords)
}
type unknownTopicProduces struct {
buffered []promisedRec
wait chan error // retriable errors
fatal chan error // must-signal quit errors; capacity 1
}
func (p *producer) init(cl *Client) {
p.cl = cl
p.topics = newTopicsPartitions()
p.unknownTopics = make(map[string]*unknownTopicProduces)
p.waitBuffer = make(chan struct{}, math.MaxInt32)
p.idVersion = -1
p.id.Store(&producerID{
id: -1,
epoch: -1,
err: errReloadProducerID,
})
p.c = sync.NewCond(&p.mu)
inithooks := func() {
if p.hooks == nil {
p.hooks = &struct {
buffered []HookProduceRecordBuffered
unbuffered []HookProduceRecordUnbuffered
}{}
}
}
cl.cfg.hooks.each(func(h Hook) {
if h, ok := h.(HookProduceRecordBuffered); ok {
inithooks()
p.hooks.buffered = append(p.hooks.buffered, h)
}
if h, ok := h.(HookProduceRecordUnbuffered); ok {
inithooks()
p.hooks.unbuffered = append(p.hooks.unbuffered, h)
}
if _, ok := h.(HookProduceBatchWritten); ok {
p.hasHookBatchWritten = true
}
})
}
func (p *producer) purgeTopics(topics []string) {
p.topicsMu.Lock()
defer p.topicsMu.Unlock()
p.unknownTopicsMu.Lock()
for _, topic := range topics {
if unknown, exists := p.unknownTopics[topic]; exists {
delete(p.unknownTopics, topic)
close(unknown.wait)
p.promiseBatch(batchPromise{
recs: unknown.buffered,
err: errPurged,
})
}
}
p.unknownTopicsMu.Unlock()
toStore := p.topics.clone()
defer p.topics.storeData(toStore)
for _, topic := range topics {
d := toStore.loadTopic(topic)
if d == nil {
continue
}
delete(toStore, topic)
for _, p := range d.partitions {
r := p.records
// First we set purged, so that anything in the process
// of being buffered will immediately fail when it goes
// to buffer.
r.mu.Lock()
r.purged = true
r.mu.Unlock()
// Now we remove from the sink. When we do, the recBuf
// is effectively abandonded. Any active produces may
// finish before we fail the records; if they finish
// after they will no longer belong in the batch, but
// they may have been produced. This is the duplicate
// risk a user runs when purging.
r.sink.removeRecBuf(r)
// Once abandonded, we now need to fail anything that
// was buffered.
go func() {
r.mu.Lock()
defer r.mu.Unlock()
r.failAllRecords(errPurged)
}()
}
}
}
func (p *producer) isAborting() bool { return atomic.LoadInt32(&p.aborting) > 0 }
func noPromise(*Record, error) {}
// ProduceResult is the result of producing a record in a synchronous manner.
type ProduceResult struct {
// Record is the produced record. It is always non-nil.
//
// If this record was produced successfully, its attrs / offset / id /
// epoch / etc. fields are filled in on return if possible (i.e. when
// producing with acks required).
Record *Record
// Err is a potential produce error. If this is non-nil, the record was
// not produced successfully.
Err error
}
// ProduceResults is a collection of produce results.
type ProduceResults []ProduceResult
// FirstErr returns the first erroring result, if any.
func (rs ProduceResults) FirstErr() error {
for _, r := range rs {
if r.Err != nil {
return r.Err
}
}
return nil
}
// First the first record and error in the produce results.
//
// This function is useful if you only passed one record to ProduceSync.
func (rs ProduceResults) First() (*Record, error) {
return rs[0].Record, rs[0].Err
}
// ProduceSync is a synchronous produce. See the Produce documentation for an
// in depth description of how producing works.
//
// This function produces all records in one range loop and waits for them all
// to be produced before returning.
func (cl *Client) ProduceSync(ctx context.Context, rs ...*Record) ProduceResults {
var (
wg sync.WaitGroup
results = make(ProduceResults, 0, len(rs))
promise = func(r *Record, err error) {
results = append(results, ProduceResult{r, err})
wg.Done()
}
)
wg.Add(len(rs))
for _, r := range rs {
cl.Produce(ctx, r, promise)
}
wg.Wait()
return results
}
// FirstErrPromise is a helper type to capture only the first failing error
// when producing a batch of records with this type's Promise function.
//
// This is useful for when you only care about any record failing, and can use
// that as a signal (i.e., to abort a batch). The AbortingFirstErrPromise
// function can be used to abort all records as soon as the first error is
// encountered. If you do not need to abort, you can use this type with no
// constructor.
//
// This is similar to using ProduceResult's FirstErr function.
type FirstErrPromise struct {
wg sync.WaitGroup
once uint32
err error
cl *Client
}
// AbortingFirstErrPromise returns a FirstErrPromise that will call the
// client's AbortBufferedRecords function if an error is encountered.
//
// This can be used to quickly exit when any error is encountered, rather than
// waiting while flushing only to discover things errored.
func AbortingFirstErrPromise(cl *Client) *FirstErrPromise {
return &FirstErrPromise{
cl: cl,
}
}
// Promise is a promise for producing that will store the first error
// encountered.
func (f *FirstErrPromise) promise(_ *Record, err error) {
defer f.wg.Done()
if err != nil && atomic.SwapUint32(&f.once, 1) == 0 {
f.err = err
if f.cl != nil {
f.wg.Add(1)
go func() {
defer f.wg.Done()
f.cl.AbortBufferedRecords(context.Background())
}()
}
}
}
// Promise returns a promise for producing that will store the first error
// encountered.
//
// The returned promise must eventually be called, because a FirstErrPromise
// does not return from 'Err' until all promises are completed.
func (f *FirstErrPromise) Promise() func(*Record, error) {
f.wg.Add(1)
return f.promise
}
// Err waits for all promises to complete and then returns any stored error.
func (f *FirstErrPromise) Err() error {
f.wg.Wait()
return f.err
}
// TryProduce is similar to Produce, but rather than blocking if the client
// currently has MaxBufferedRecords buffered, this fails immediately with
// ErrMaxBuffered. See the Produce documentation for more details.
func (cl *Client) TryProduce(
ctx context.Context,
r *Record,
promise func(*Record, error),
) {
cl.produce(ctx, r, promise, false)
}
// Produce sends a Kafka record to the topic in the record's Topic field,
// calling an optional `promise` with the record and a potential error when
// Kafka replies. For a synchronous produce, see ProduceSync. Records are
// produced in order per partition if the record is produced successfully.
// Successfully produced records will have their attributes, offset, and
// partition set before the promise is called. All promises are called serially
// (and should be relatively fast). If a record's timestamp is unset, this
// sets the timestamp to time.Now.
//
// If the topic field is empty, the client will use the DefaultProduceTopic; if
// that is also empty, the record is failed immediately. If the record is too
// large to fit in a batch on its own in a produce request, the record will be
// failed with immediately kerr.MessageTooLarge.
//
// If the client is configured to automatically flush the client currently has
// the configured maximum amount of records buffered, Produce will block. The
// context can be used to cancel waiting while records flush to make space. In
// contrast, if flushing is configured, the record will be failed immediately
// with ErrMaxBuffered (this same behavior can be had with TryProduce).
//
// Once a record is buffered into a batch, it can be canceled in three ways:
// canceling the context, the record timing out, or hitting the maximum
// retries. If any of these conditions are hit and it is currently safe to fail
// records, all buffered records for the relevant partition are failed. Only
// the first record's context in a batch is considered when determining whether
// the batch should be canceled.
//
// If the client is transactional and a transaction has not been begun, the
// promise is immediately called with an error corresponding to not being in a
// transaction.
func (cl *Client) Produce(
ctx context.Context,
r *Record,
promise func(*Record, error),
) {
cl.produce(ctx, r, promise, true)
}
func (cl *Client) produce(
ctx context.Context,
r *Record,
promise func(*Record, error),
block bool,
) {
if ctx == nil {
ctx = context.Background()
}
if r.Context == nil {
r.Context = ctx
}
if promise == nil {
promise = noPromise
}
p := &cl.producer
if p.hooks != nil {
for _, h := range p.hooks.buffered {
h.OnProduceRecordBuffered(r)
}
}
if atomic.AddInt64(&p.bufferedRecords, 1) > cl.cfg.maxBufferedRecords {
// If the client ctx cancels or the produce ctx cancels, we
// need to un-count our buffering of this record. We also need
// to drain a slot from the waitBuffer chan, which could be
// sent to right when we are erroring.
drainBuffered := func(err error) {
p.promiseRecord(promisedRec{ctx, promise, r}, err)
<-p.waitBuffer
}
if !block || cl.cfg.manualFlushing {
drainBuffered(ErrMaxBuffered)
return
}
select {
case <-p.waitBuffer:
case <-cl.ctx.Done():
drainBuffered(ErrClientClosed)
return
case <-ctx.Done():
drainBuffered(ctx.Err())
return
}
}
// Neither of the errors below should be hit in applications.
if r.Topic == "" {
def := cl.cfg.defaultProduceTopic
if def == "" {
p.promiseRecord(promisedRec{ctx, promise, r}, errNoTopic)
return
}
r.Topic = def
}
if cl.cfg.txnID != nil && atomic.LoadUint32(&p.producingTxn) != 1 {
p.promiseRecord(promisedRec{ctx, promise, r}, errNotInTransaction)
return
}
cl.partitionRecord(promisedRec{ctx, promise, r})
}
type batchPromise struct {
baseOffset int64
pid int64
epoch int16
attrs RecordAttrs
partition int32
recs []promisedRec
err error
}
func (p *producer) promiseBatch(b batchPromise) {
if first := p.batchPromises.push(b); first {
go p.finishPromises(b)
}
}
func (p *producer) promiseRecord(pr promisedRec, err error) {
p.promiseBatch(batchPromise{recs: []promisedRec{pr}, err: err})
}
func (p *producer) finishPromises(b batchPromise) {
cl := p.cl
var more bool
start:
p.promisesMu.Lock()
for i, pr := range b.recs {
pr.Offset = b.baseOffset + int64(i)
pr.Partition = b.partition
pr.ProducerID = b.pid
pr.ProducerEpoch = b.epoch
pr.Attrs = b.attrs
cl.finishRecordPromise(pr, b.err)
b.recs[i] = promisedRec{}
}
p.promisesMu.Unlock()
if cap(b.recs) > 4 {
cl.prsPool.put(b.recs)
}
b, more = p.batchPromises.dropPeek()
if more {
goto start
}
}
func (cl *Client) finishRecordPromise(pr promisedRec, err error) {
p := &cl.producer
if p.hooks != nil {
for _, h := range p.hooks.unbuffered {
h.OnProduceRecordUnbuffered(pr.Record, err)
}
}
// We call the promise before finishing the record; this allows users
// of Flush to know that all buffered records are completely done
// before Flush returns.
pr.promise(pr.Record, err)
buffered := atomic.AddInt64(&p.bufferedRecords, -1)
if buffered >= cl.cfg.maxBufferedRecords {
p.waitBuffer <- struct{}{}
} else if buffered == 0 && atomic.LoadInt32(&p.flushing) > 0 {
p.mu.Lock()
p.mu.Unlock() //nolint:gocritic,staticcheck // We use the lock as a barrier, unlocking immediately is safe.
p.c.Broadcast()
}
}
// partitionRecord loads the partitions for a topic and produce to them. If
// the topic does not currently exist, the record is buffered in unknownTopics
// for a metadata update to deal with.
func (cl *Client) partitionRecord(pr promisedRec) {
parts, partsData := cl.partitionsForTopicProduce(pr)
if parts == nil { // saved in unknownTopics
return
}
cl.doPartitionRecord(parts, partsData, pr)
}
// doPartitionRecord is separate so that metadata updates that load unknown
// partitions can call this directly.
func (cl *Client) doPartitionRecord(parts *topicPartitions, partsData *topicPartitionsData, pr promisedRec) {
if partsData.loadErr != nil && !kerr.IsRetriable(partsData.loadErr) {
cl.producer.promiseRecord(pr, partsData.loadErr)
return
}
parts.partsMu.Lock()
defer parts.partsMu.Unlock()
if parts.partitioner == nil {
parts.partitioner = cl.cfg.partitioner.ForTopic(pr.Topic)
}
mapping := partsData.writablePartitions
if parts.partitioner.RequiresConsistency(pr.Record) {
mapping = partsData.partitions
}
if len(mapping) == 0 {
cl.producer.promiseRecord(pr, errors.New("unable to partition record due to no usable partitions"))
return
}
var pick int
tlp, _ := parts.partitioner.(TopicBackupPartitioner)
if tlp != nil {
if parts.lb == nil {
parts.lb = new(leastBackupInput)
}
parts.lb.mapping = mapping
pick = tlp.PartitionByBackup(pr.Record, len(mapping), parts.lb)
} else {
pick = parts.partitioner.Partition(pr.Record, len(mapping))
}
if pick < 0 || pick >= len(mapping) {
cl.producer.promiseRecord(pr, fmt.Errorf("invalid record partitioning choice of %d from %d available", pick, len(mapping)))
return
}
partition := mapping[pick]
onNewBatch, _ := parts.partitioner.(TopicPartitionerOnNewBatch)
abortOnNewBatch := onNewBatch != nil
processed := partition.records.bufferRecord(pr, abortOnNewBatch) // KIP-480
if !processed {
onNewBatch.OnNewBatch()
if tlp != nil {
parts.lb.mapping = mapping
pick = tlp.PartitionByBackup(pr.Record, len(mapping), parts.lb)
} else {
pick = parts.partitioner.Partition(pr.Record, len(mapping))
}
if pick < 0 || pick >= len(mapping) {
cl.producer.promiseRecord(pr, fmt.Errorf("invalid record partitioning choice of %d from %d available", pick, len(mapping)))
return
}
partition = mapping[pick]
partition.records.bufferRecord(pr, false) // KIP-480
}
}
// ProducerID returns, loading if necessary, the current producer ID and epoch.
// This returns an error if the producer ID could not be loaded, if the
// producer ID has fatally errored, or if the context is canceled.
func (cl *Client) ProducerID(ctx context.Context) (int64, int16, error) {
var (
id int64
epoch int16
err error
done = make(chan struct{})
)
go func() {
defer close(done)
id, epoch, err = cl.producerID()
}()
select {
case <-ctx.Done():
return 0, 0, ctx.Err()
case <-done:
return id, epoch, err
}
}
type producerID struct {
id int64
epoch int16
err error
}
var errReloadProducerID = errors.New("producer id needs reloading")
// initProducerID initializes the client's producer ID for idempotent
// producing only (no transactions, which are more special). After the first
// load, this clears all buffered unknown topics.
func (cl *Client) producerID() (int64, int16, error) {
p := &cl.producer
id := p.id.Load().(*producerID)
if errors.Is(id.err, errReloadProducerID) {
p.idMu.Lock()
defer p.idMu.Unlock()
if id = p.id.Load().(*producerID); errors.Is(id.err, errReloadProducerID) {
if cl.cfg.disableIdempotency {
cl.cfg.logger.Log(LogLevelInfo, "skipping producer id initialization because the client was configured to disable idempotent writes")
id = &producerID{
id: -1,
epoch: -1,
err: nil,
}
p.id.Store(id)
} else if cl.cfg.txnID == nil && id.id >= 0 && id.epoch < math.MaxInt16-1 {
// For the idempotent producer, as specified in KIP-360,
// if we had an ID, we can bump the epoch locally.
// If we are at the max epoch, we will ask for a new ID.
cl.resetAllProducerSequences()
id = &producerID{
id: id.id,
epoch: id.epoch + 1,
err: nil,
}
p.id.Store(id)
} else {
newID, keep := cl.doInitProducerID(id.id, id.epoch)
if keep {
id = newID
// Whenever we have a new producer ID, we need
// our sequence numbers to be 0. On the first
// record produced, this will be true, but if
// we were signaled to reset the producer ID,
// then we definitely still need to reset here.
cl.resetAllProducerSequences()
p.id.Store(id)
} else {
// If we are not keeping the producer ID,
// we will return our old ID but with a
// static error that we can check or bubble
// up where needed.
id = &producerID{
id: id.id,
epoch: id.epoch,
err: errProducerIDLoadFail,
}
}
}
}
}
return id.id, id.epoch, id.err
}
// As seen in KAFKA-12152, if we bump an epoch, we have to reset sequence nums
// for every partition. Otherwise, we will use a new id/epoch for a partition
// and trigger OOOSN errors.
//
// Pre 2.5, this function is only be called if it is acceptable to continue
// on data loss (idempotent producer with no StopOnDataLoss option).
//
// 2.5+, it is safe to call this if the producer ID can be reset (KIP-360),
// in EndTransaction.
func (cl *Client) resetAllProducerSequences() {
for _, tp := range cl.producer.topics.load() {
for _, p := range tp.load().partitions {
p.records.mu.Lock()
p.records.needSeqReset = true
p.records.mu.Unlock()
}
}
}
func (cl *Client) failProducerID(id int64, epoch int16, err error) {
p := &cl.producer
p.idMu.Lock()
defer p.idMu.Unlock()
current := p.id.Load().(*producerID)
if current.id != id || current.epoch != epoch {
cl.cfg.logger.Log(LogLevelInfo, "ignoring a fail producer id request due to current id being different",
"current_id", current.id,
"current_epoch", current.epoch,
"current_err", current.err,
"fail_id", id,
"fail_epoch", epoch,
"fail_err", err,
)
return // failed an old id
}
// If this is not UnknownProducerID, then we cannot recover production.
//
// If this is UnknownProducerID without a txnID, then we are here from
// stopOnDataLoss in sink.go (see large comment there).
//
// If this is UnknownProducerID with a txnID, then EndTransaction will
// recover us.
p.id.Store(&producerID{
id: id,
epoch: epoch,
err: err,
})
}
// doInitProducerID inits the idempotent ID and potentially the transactional
// producer epoch, returning whether to keep the result.
func (cl *Client) doInitProducerID(lastID int64, lastEpoch int16) (*producerID, bool) {
cl.cfg.logger.Log(LogLevelInfo, "initializing producer id")
req := kmsg.NewPtrInitProducerIDRequest()
req.TransactionalID = cl.cfg.txnID
req.ProducerID = lastID
req.ProducerEpoch = lastEpoch
if cl.cfg.txnID != nil {
req.TransactionTimeoutMillis = int32(cl.cfg.txnTimeout.Milliseconds())
}
resp, err := req.RequestWith(cl.ctx, cl)
if err != nil {
if errors.Is(err, errUnknownRequestKey) || errors.Is(err, errBrokerTooOld) {
cl.cfg.logger.Log(LogLevelInfo, "unable to initialize a producer id because the broker is too old or the client is pinned to an old version, continuing without a producer id")
return &producerID{-1, -1, nil}, true
}
if errors.Is(err, errChosenBrokerDead) {
select {
case <-cl.ctx.Done():
cl.cfg.logger.Log(LogLevelInfo, "producer id initialization failure due to dying client", "err", err)
return &producerID{lastID, lastEpoch, ErrClientClosed}, true
default:
}
}
cl.cfg.logger.Log(LogLevelInfo, "producer id initialization failure, discarding initialization attempt", "err", err)
return &producerID{lastID, lastEpoch, err}, false
}
if err = kerr.ErrorForCode(resp.ErrorCode); err != nil {
// We could receive concurrent transactions; this is ignorable
// and we just want to re-init.
if kerr.IsRetriable(err) || errors.Is(err, kerr.ConcurrentTransactions) {
cl.cfg.logger.Log(LogLevelInfo, "producer id initialization resulted in retriable error, discarding initialization attempt", "err", err)
return &producerID{lastID, lastEpoch, err}, false
}
cl.cfg.logger.Log(LogLevelInfo, "producer id initialization errored", "err", err)
return &producerID{lastID, lastEpoch, err}, true
}
cl.cfg.logger.Log(LogLevelInfo, "producer id initialization success", "id", resp.ProducerID, "epoch", resp.ProducerEpoch)
// We track if this was v3. We do not need to gate this behind a mutex,
// because the only other use is EndTransaction's read, which is
// documented to only be called sequentially after producing.
if cl.producer.idVersion == -1 {
cl.producer.idVersion = req.Version
}
return &producerID{resp.ProducerID, resp.ProducerEpoch, nil}, true
}
// partitionsForTopicProduce returns the topic partitions for a record.
// If the topic is not loaded yet, this buffers the record and returns
// nil, nil.
func (cl *Client) partitionsForTopicProduce(pr promisedRec) (*topicPartitions, *topicPartitionsData) {
p := &cl.producer
topic := pr.Topic
topics := p.topics.load()
parts, exists := topics[topic]
if exists {
if v := parts.load(); len(v.partitions) > 0 {
return parts, v
}
}
if !exists { // topic did not exist: check again under mu and potentially create it
p.topicsMu.Lock()
defer p.topicsMu.Unlock()
if parts, exists = p.topics.load()[topic]; !exists { // update parts for below
// Before we store the new topic, we lock unknown
// topics to prevent a concurrent metadata update
// seeing our new topic before we are waiting from the
// addUnknownTopicRecord fn. Otherwise, we would wait
// and never be re-notified.
p.unknownTopicsMu.Lock()
defer p.unknownTopicsMu.Unlock()
p.topics.storeTopics([]string{topic})
cl.addUnknownTopicRecord(pr)
cl.triggerUpdateMetadataNow("forced load because we are producing to a new topic for the first time")
return nil, nil
}
}
// Here, the topic existed, but maybe has not loaded partitions yet. We
// have to lock unknown topics first to ensure ordering just in case a
// load has not happened.
p.unknownTopicsMu.Lock()
defer p.unknownTopicsMu.Unlock()
if v := parts.load(); len(v.partitions) > 0 {
return parts, v
}
cl.addUnknownTopicRecord(pr)
cl.triggerUpdateMetadata(false, "reload trigger due to produce topic still not known")
return nil, nil // our record is buffered waiting for metadata update; nothing to return
}
// addUnknownTopicRecord adds a record to a topic whose partitions are
// currently unknown. This is always called with the unknownTopicsMu held.
func (cl *Client) addUnknownTopicRecord(pr promisedRec) {
unknown := cl.producer.unknownTopics[pr.Topic]
if unknown == nil {
unknown = &unknownTopicProduces{
buffered: make([]promisedRec, 0, 100),
wait: make(chan error, 5),
fatal: make(chan error, 1),
}
cl.producer.unknownTopics[pr.Topic] = unknown
}
unknown.buffered = append(unknown.buffered, pr)
if len(unknown.buffered) == 1 {
go cl.waitUnknownTopic(pr.ctx, pr.Topic, unknown)
}
}
// waitUnknownTopic waits for a notification
func (cl *Client) waitUnknownTopic(
rctx context.Context,
topic string,
unknown *unknownTopicProduces,
) {
cl.cfg.logger.Log(LogLevelInfo, "producing to a new topic for the first time, fetching metadata to learn its partitions", "topic", topic)
var (
tries int
unknownTries int64
err error
after <-chan time.Time
)
if timeout := cl.cfg.recordTimeout; timeout > 0 {
timer := time.NewTimer(cl.cfg.recordTimeout)
defer timer.Stop()
after = timer.C
}
// Ordering: aborting is set first, then unknown topics are manually
// canceled in a lock. New unknown topics after that lock will see
// aborting here and immediately cancel themselves.
if cl.producer.isAborting() {
err = ErrAborting
}
for err == nil {
select {
case <-rctx.Done():
err = rctx.Err()
case <-cl.ctx.Done():
err = ErrClientClosed
case <-after:
err = ErrRecordTimeout
case err = <-unknown.fatal:
case retriableErr, ok := <-unknown.wait:
if !ok {
cl.cfg.logger.Log(LogLevelInfo, "done waiting for metadata for new topic", "topic", topic)
return // metadata was successful!
}
cl.cfg.logger.Log(LogLevelInfo, "new topic metadata wait failed, retrying wait", "topic", topic, "err", retriableErr)
tries++
if int64(tries) >= cl.cfg.recordRetries {
err = fmt.Errorf("no partitions available after attempting to refresh metadata %d times, last err: %w", tries, retriableErr)
}
if cl.cfg.maxUnknownFailures >= 0 && errors.Is(retriableErr, kerr.UnknownTopicOrPartition) {
unknownTries++
if unknownTries > cl.cfg.maxUnknownFailures {
err = retriableErr
}
}
}
}
// If we errored above, we come down here to potentially clear the
// topic wait and fail all buffered records. However, under some
// extreme conditions, a quickly following metadata update could delete
// our unknown topic, and then a produce could recreate a new unknown
// topic. We only delete and finish promises if the pointer in the
// unknown topic map is still the same.
p := &cl.producer
p.unknownTopicsMu.Lock()
defer p.unknownTopicsMu.Unlock()
nowUnknown := p.unknownTopics[topic]
if nowUnknown != unknown {
return
}
cl.cfg.logger.Log(LogLevelInfo, "new topic metadata wait failed, done retrying, failing all records", "topic", topic, "err", err)
delete(p.unknownTopics, topic)
p.promiseBatch(batchPromise{
recs: unknown.buffered,
err: err,
})
}
// Flush hangs waiting for all buffered records to be flushed, stopping all
// lingers if necessary.
//
// If the context finishes (Done), this returns the context's error.
//
// This function is safe to call multiple times concurrently, and safe to call
// concurrent with Flush.
func (cl *Client) Flush(ctx context.Context) error {
p := &cl.producer
// Signal to finishRecord that we want to be notified once buffered hits 0.
// Also forbid any new producing to start a linger.
atomic.AddInt32(&p.flushing, 1)
defer atomic.AddInt32(&p.flushing, -1)
cl.cfg.logger.Log(LogLevelInfo, "flushing")
defer cl.cfg.logger.Log(LogLevelDebug, "flushed")
// At this point, if lingering is configured, nothing will _start_ a
// linger because the producer's flushing atomic int32 is nonzero. We
// must wake anything that could be lingering up, after which all sinks
// will loop draining.
if cl.cfg.linger > 0 || cl.cfg.manualFlushing {
for _, parts := range p.topics.load() {
for _, part := range parts.load().partitions {
part.records.unlingerAndManuallyDrain()
}
}
}
quit := false
done := make(chan struct{})
go func() {
p.mu.Lock()
defer p.mu.Unlock()
defer close(done)
for !quit && atomic.LoadInt64(&p.bufferedRecords) > 0 {
p.c.Wait()
}
}()
select {
case <-done:
return nil
case <-ctx.Done():
p.mu.Lock()
quit = true
p.mu.Unlock()
p.c.Broadcast()
return ctx.Err()
}
}
func (p *producer) pause(ctx context.Context) error {
atomic.AddInt64(&p.inflight, 1<<48)
quit := false
done := make(chan struct{})
go func() {
p.mu.Lock()
defer p.mu.Unlock()
defer close(done)
for !quit && atomic.LoadInt64(&p.inflight)&((1<<48)-1) != 0 {
p.c.Wait()
}
}()
select {
case <-done:
return nil
case <-ctx.Done():
p.mu.Lock()
quit = true
p.mu.Unlock()
p.c.Broadcast()
p.resume() // dec our inflight
return ctx.Err()
}
}
func (p *producer) resume() {
if atomic.AddInt64(&p.inflight, -1<<48) == 0 {
p.cl.allSinksAndSources(func(sns sinkAndSource) {
sns.sink.maybeDrain()
})
}
}
func (p *producer) maybeAddInflight() bool {