forked from twmb/franz-go
-
Notifications
You must be signed in to change notification settings - Fork 1
/
sink.go
2206 lines (1945 loc) · 68 KB
/
sink.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 (
"bytes"
"context"
"errors"
"fmt"
"hash/crc32"
"math"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/twmb/franz-go/pkg/kbin"
"github.com/twmb/franz-go/pkg/kerr"
"github.com/twmb/franz-go/pkg/kmsg"
)
type sink struct {
cl *Client // our owning client, for cfg, metadata triggering, context, etc.
nodeID int32 // the node ID of the broker this sink belongs to
// inflightSem controls the number of concurrent produce requests. We
// start with a limit of 1, which covers Kafka v0.11.0. On the first
// response, we check what version was set in the request. If it is at
// least 4, which 1.0 introduced, we upgrade the sem size.
inflightSem atomic.Value
produceVersion atomicI32 // negative is unset, positive is version
drainState workLoop
// seqRespsMu, guarded by seqRespsMu, contains responses that must
// be handled sequentially. These responses are handled asynchronously,
// but sequentially.
seqResps ringSeqResp
backoffMu sync.Mutex // guards the following
needBackoff bool
backoffSeq uint32 // prevents pile on failures
// consecutiveFailures is incremented every backoff and cleared every
// successful response. For simplicity, if we have a good response
// following an error response before the error response's backoff
// occurs, the backoff is not cleared.
consecutiveFailures atomicU32
recBufsMu sync.Mutex // guards the following
recBufs []*recBuf // contains all partition records for batch building
recBufsStart int // incremented every req to avoid large batch starvation
}
type seqResp struct {
resp kmsg.Response
err error
done chan struct{}
br *broker
promise func(*broker, kmsg.Response, error)
}
func (cl *Client) newSink(nodeID int32) *sink {
s := &sink{
cl: cl,
nodeID: nodeID,
}
s.produceVersion.Store(-1)
maxInflight := 1
if cl.cfg.disableIdempotency {
maxInflight = cl.cfg.maxProduceInflight
}
s.inflightSem.Store(make(chan struct{}, maxInflight))
return s
}
// createReq returns a produceRequest from currently buffered records
// and whether there are more records to create more requests immediately.
func (s *sink) createReq(id int64, epoch int16) (*produceRequest, *kmsg.AddPartitionsToTxnRequest, bool) {
req := &produceRequest{
txnID: s.cl.cfg.txnID,
acks: s.cl.cfg.acks.val,
timeout: int32(s.cl.cfg.produceTimeout.Milliseconds()),
batches: make(seqRecBatches, 5),
producerID: id,
producerEpoch: epoch,
hasHook: s.cl.producer.hasHookBatchWritten,
compressor: s.cl.compressor,
wireLength: s.cl.baseProduceRequestLength(), // start length with no topics
wireLengthLimit: s.cl.cfg.maxBrokerWriteBytes,
}
txnBuilder := txnReqBuilder{
txnID: req.txnID,
id: id,
epoch: epoch,
}
var moreToDrain bool
s.recBufsMu.Lock()
defer s.recBufsMu.Unlock()
recBufsIdx := s.recBufsStart
for i := 0; i < len(s.recBufs); i++ {
recBuf := s.recBufs[recBufsIdx]
recBufsIdx = (recBufsIdx + 1) % len(s.recBufs)
recBuf.mu.Lock()
if recBuf.failing || len(recBuf.batches) == recBuf.batchDrainIdx || recBuf.inflightOnSink != nil && recBuf.inflightOnSink != s || recBuf.inflight != 0 && !recBuf.okOnSink {
recBuf.mu.Unlock()
continue
}
batch := recBuf.batches[recBuf.batchDrainIdx]
if added := req.tryAddBatch(s.produceVersion.Load(), recBuf, batch); !added {
recBuf.mu.Unlock()
moreToDrain = true
continue
}
recBuf.inflightOnSink = s
recBuf.inflight++
recBuf.batchDrainIdx++
recBuf.seq = incrementSequence(recBuf.seq, int32(len(batch.records)))
moreToDrain = moreToDrain || recBuf.tryStopLingerForDraining()
recBuf.mu.Unlock()
txnBuilder.add(recBuf)
}
// We could have lost our only record buffer just before we grabbed the
// lock above, so we have to check there are recBufs.
if len(s.recBufs) > 0 {
s.recBufsStart = (s.recBufsStart + 1) % len(s.recBufs)
}
return req, txnBuilder.req, moreToDrain
}
func incrementSequence(sequence, increment int32) int32 {
if sequence > math.MaxInt32-increment {
return increment - (math.MaxInt32 - sequence) - 1
}
return sequence + increment
}
type txnReqBuilder struct {
txnID *string
req *kmsg.AddPartitionsToTxnRequest
id int64
epoch int16
addedTopics map[string]int // topic => index into req
}
func (t *txnReqBuilder) add(rb *recBuf) {
if t.txnID == nil {
return
}
if rb.addedToTxn.Swap(true) {
return
}
if t.req == nil {
req := kmsg.NewPtrAddPartitionsToTxnRequest()
req.TransactionalID = *t.txnID
req.ProducerID = t.id
req.ProducerEpoch = t.epoch
t.req = req
t.addedTopics = make(map[string]int, 10)
}
idx, exists := t.addedTopics[rb.topic]
if !exists {
idx = len(t.req.Topics)
t.addedTopics[rb.topic] = idx
reqTopic := kmsg.NewAddPartitionsToTxnRequestTopic()
reqTopic.Topic = rb.topic
t.req.Topics = append(t.req.Topics, reqTopic)
}
t.req.Topics[idx].Partitions = append(t.req.Topics[idx].Partitions, rb.partition)
}
func (s *sink) maybeDrain() {
if s.cl.cfg.manualFlushing && s.cl.producer.flushing.Load() == 0 {
return
}
if s.drainState.maybeBegin() {
go s.drain()
}
}
func (s *sink) maybeBackoff() {
s.backoffMu.Lock()
backoff := s.needBackoff
s.backoffMu.Unlock()
if !backoff {
return
}
defer s.clearBackoff()
s.cl.triggerUpdateMetadata(false, "opportunistic load during sink backoff") // as good a time as any
tries := int(s.consecutiveFailures.Add(1))
after := time.NewTimer(s.cl.cfg.retryBackoff(tries))
defer after.Stop()
select {
case <-after.C:
case <-s.cl.ctx.Done():
}
}
func (s *sink) maybeTriggerBackoff(seq uint32) {
s.backoffMu.Lock()
defer s.backoffMu.Unlock()
if seq == s.backoffSeq {
s.needBackoff = true
}
}
func (s *sink) clearBackoff() {
s.backoffMu.Lock()
defer s.backoffMu.Unlock()
s.backoffSeq++
s.needBackoff = false
}
// drain drains buffered records and issues produce requests.
//
// This function is harmless if there are no records that need draining.
// We rely on that to not worry about accidental triggers of this function.
func (s *sink) drain() {
again := true
for again {
s.maybeBackoff()
sem := s.inflightSem.Load().(chan struct{})
select {
case sem <- struct{}{}:
case <-s.cl.ctx.Done():
s.drainState.hardFinish()
return
}
again = s.drainState.maybeFinish(s.produce(sem))
}
}
func (s *sink) produce(sem <-chan struct{}) bool {
var produced bool
defer func() {
if !produced {
<-sem
}
}()
// We could have been triggered from a metadata update even though the
// user is not producing at all. If we have no buffered records, let's
// avoid potentially creating a producer ID.
if s.cl.producer.bufferedRecords.Load() == 0 {
return false
}
// producerID can fail from:
// - retry failure
// - auth failure
// - transactional: a produce failure that failed the producer ID
// - AddPartitionsToTxn failure (see just below)
//
// All but the first error is fatal. Recovery may be possible with
// EndTransaction in specific cases, but regardless, all buffered
// records must fail.
//
// NOTE: we init the producer ID before creating a request to ensure we
// are always using the latest id/epoch with the proper sequence
// numbers. (i.e., resetAllSequenceNumbers && producerID logic combo).
id, epoch, err := s.cl.producerID()
if err != nil {
switch {
case errors.Is(err, errProducerIDLoadFail):
s.cl.bumpRepeatedLoadErr(err)
s.cl.cfg.logger.Log(LogLevelWarn, "unable to load producer ID, bumping client's buffered record load errors by 1 and retrying")
return true // whatever caused our produce, we did nothing, so keep going
case errors.Is(err, ErrClientClosed):
s.cl.failBufferedRecords(err)
default:
s.cl.cfg.logger.Log(LogLevelError, "fatal InitProducerID error, failing all buffered records", "broker", logID(s.nodeID), "err", err)
s.cl.failBufferedRecords(err)
}
return false
}
if !s.cl.producer.maybeAddInflight() { // must do before marking recBufs on a txn
return false
}
defer func() {
if !produced {
s.cl.producer.decInflight()
}
}()
// NOTE: we create the req AFTER getting our producer ID!
//
// If a prior response caused errReloadProducerID, then calling
// producerID() sets needSeqReset, and creating the request resets
// sequence numbers. We need to have that logic occur before we create
// the request, otherwise we will create a request with the old
// sequence numbers using our new producer ID, which will then again
// fail with OOOSN.
req, txnReq, moreToDrain := s.createReq(id, epoch)
if len(req.batches) == 0 { // everything was failing or lingering
return moreToDrain
}
if txnReq != nil {
// txnReq can fail from:
// - retry failure
// - auth failure
// - producer id mapping / epoch errors
// The latter case can potentially recover with the kip logic
// we have defined in EndTransaction. Regardless, on failure
// here, all buffered records must fail.
// We do not need to clear the addedToTxn flag for any recBuf
// it was set on, since producer id recovery resets the flag.
batchesStripped, err := s.doTxnReq(req, txnReq)
if err != nil {
switch {
case isRetryableBrokerErr(err) || isDialNonTimeoutErr(err):
s.cl.bumpRepeatedLoadErr(err)
s.cl.cfg.logger.Log(LogLevelWarn, "unable to AddPartitionsToTxn due to retryable broker err, bumping client's buffered record load errors by 1 and retrying", "err", err)
s.cl.triggerUpdateMetadata(false, "attempting to refresh broker list due to failed AddPartitionsToTxn requests")
return moreToDrain || len(req.batches) > 0 // nothing stripped if request-issuing error
default:
// Note that err can be InvalidProducerEpoch, which is
// potentially recoverable in EndTransaction.
//
// We do not fail all buffered records here,
// because that can lead to undesirable behavior
// with produce request vs. end txn (KAFKA-12671)
s.cl.failProducerID(id, epoch, err)
s.cl.cfg.logger.Log(LogLevelError, "fatal AddPartitionsToTxn error, failing all buffered records (it is possible the client can recover after EndTransaction)", "broker", logID(s.nodeID), "err", err)
}
return false
}
// If we stripped everything, ensure we backoff to force a
// metadata load. If not everything was stripped, we issue our
// request and ensure we will retry a producing until
// everything is stripped (and we eventually back off).
if batchesStripped {
moreToDrain = true
if len(req.batches) == 0 {
s.maybeTriggerBackoff(s.backoffSeq)
}
}
}
if len(req.batches) == 0 { // txn req could have removed some partitions to retry later (unknown topic, etc.)
return moreToDrain
}
req.backoffSeq = s.backoffSeq // safe to read outside mu since we are in drain loop
produced = true
batches := req.batches.sliced()
s.doSequenced(req, func(br *broker, resp kmsg.Response, err error) {
s.handleReqResp(br, req, resp, err)
s.cl.producer.decInflight()
batches.eachOwnerLocked((*recBatch).decInflight)
<-sem
})
return moreToDrain
}
// With handleSeqResps below, this function ensures that all request responses
// are handled in order. We use this guarantee while in handleReqResp below.
func (s *sink) doSequenced(
req kmsg.Request,
promise func(*broker, kmsg.Response, error),
) {
wait := &seqResp{
done: make(chan struct{}),
promise: promise,
}
br, err := s.cl.brokerOrErr(s.cl.ctx, s.nodeID, errUnknownBroker)
if err != nil {
wait.err = err
close(wait.done)
} else {
br.do(s.cl.ctx, req, func(resp kmsg.Response, err error) {
wait.resp = resp
wait.err = err
close(wait.done)
})
wait.br = br
}
if first := s.seqResps.push(wait); first {
go s.handleSeqResps(wait)
}
}
// Ensures that all request responses are processed in order.
func (s *sink) handleSeqResps(wait *seqResp) {
var more bool
start:
<-wait.done
wait.promise(wait.br, wait.resp, wait.err)
wait, more = s.seqResps.dropPeek()
if more {
goto start
}
}
// Issues an AddPartitionsToTxnRequest before a produce request for all
// partitions that need to be added to a transaction.
func (s *sink) doTxnReq(
req *produceRequest,
txnReq *kmsg.AddPartitionsToTxnRequest,
) (stripped bool, err error) {
// If we return an unretryable error, then we have to reset everything
// to not be in the transaction and begin draining at the start.
//
// These batches must be the first in their recBuf, because we would
// not be trying to add them to a partition if they were not.
defer func() {
if err != nil {
req.batches.eachOwnerLocked(seqRecBatch.removeFromTxn)
}
}()
err = s.cl.doWithConcurrentTransactions(s.cl.ctx, "AddPartitionsToTxn", func() error {
stripped, err = s.issueTxnReq(req, txnReq)
return err
})
return stripped, err
}
// Removing a batch from the transaction means we will not be issuing it
// inflight, and that it was not added to the txn and that we need to reset the
// drain index.
func (b *recBatch) removeFromTxn() {
b.owner.addedToTxn.Store(false)
b.owner.resetBatchDrainIdx()
b.decInflight()
}
func (s *sink) issueTxnReq(
req *produceRequest,
txnReq *kmsg.AddPartitionsToTxnRequest,
) (stripped bool, fatalErr error) {
resp, err := txnReq.RequestWith(s.cl.ctx, s.cl)
if err != nil {
return false, err
}
for _, topic := range resp.Topics {
topicBatches, ok := req.batches[topic.Topic]
if !ok {
s.cl.cfg.logger.Log(LogLevelError, "broker replied with topic in AddPartitionsToTxnResponse that was not in request", "topic", topic.Topic)
continue
}
for _, partition := range topic.Partitions {
if err := kerr.ErrorForCode(partition.ErrorCode); err != nil {
// OperationNotAttempted is set for all partitions that are authorized
// if any partition is unauthorized _or_ does not exist. We simply remove
// unattempted partitions and treat them as retryable.
if !kerr.IsRetriable(err) && !errors.Is(err, kerr.OperationNotAttempted) {
fatalErr = err // auth err, etc
continue
}
batch, ok := topicBatches[partition.Partition]
if !ok {
s.cl.cfg.logger.Log(LogLevelError, "broker replied with partition in AddPartitionsToTxnResponse that was not in request", "topic", topic.Topic, "partition", partition.Partition)
continue
}
// We are stripping this retryable-err batch from the request,
// so we must reset that it has been added to the txn.
batch.owner.mu.Lock()
batch.removeFromTxn()
batch.owner.mu.Unlock()
stripped = true
delete(topicBatches, partition.Partition)
}
if len(topicBatches) == 0 {
delete(req.batches, topic.Topic)
}
}
}
return stripped, fatalErr
}
// firstRespCheck is effectively a sink.Once. On the first response, if the
// used request version is at least 4, we upgrade our inflight sem.
//
// Starting on version 4, Kafka allowed five inflight requests while
// maintaining idempotency. Before, only one was allowed.
//
// We go through an atomic because drain can be waiting on the sem (with
// capacity one). We store four here, meaning new drain loops will load the
// higher capacity sem without read/write pointer racing a current loop.
//
// This logic does mean that we will never use the full potential 5 in flight
// outside of a small window during the store, but some pages in the Kafka
// confluence basically show that more than two in flight has marginal benefit
// anyway (although that may be due to their Java API).
//
// https://cwiki.apache.org/confluence/display/KAFKA/An+analysis+of+the+impact+of+max.in.flight.requests.per.connection+and+acks+on+Producer+performance
// https://issues.apache.org/jira/browse/KAFKA-5494
func (s *sink) firstRespCheck(idempotent bool, version int16) {
if s.produceVersion.Load() < 0 {
s.produceVersion.Store(int32(version))
if idempotent && version >= 4 {
s.inflightSem.Store(make(chan struct{}, 4))
}
}
}
// handleReqClientErr is called when the client errors before receiving a
// produce response.
func (s *sink) handleReqClientErr(req *produceRequest, err error) {
switch {
default:
s.cl.cfg.logger.Log(LogLevelWarn, "random error while producing, requeueing unattempted request", "broker", logID(s.nodeID), "err", err)
fallthrough
case errors.Is(err, errUnknownBroker),
isDialNonTimeoutErr(err),
isRetryableBrokerErr(err):
updateMeta := !isRetryableBrokerErr(err)
if updateMeta {
s.cl.cfg.logger.Log(LogLevelInfo, "produce request failed, triggering metadata update", "broker", logID(s.nodeID), "err", err)
}
s.handleRetryBatches(req.batches, req.backoffSeq, updateMeta, false, "failed produce request triggered metadata update")
case errors.Is(err, ErrClientClosed):
s.cl.failBufferedRecords(ErrClientClosed)
}
}
// No acks mean no response. The following block is basically an extremely
// condensed version of the logic in handleReqResp.
func (s *sink) handleReqRespNoack(b *bytes.Buffer, debug bool, req *produceRequest) {
if debug {
fmt.Fprintf(b, "noack ")
}
for topic, partitions := range req.batches {
if debug {
fmt.Fprintf(b, "%s[", topic)
}
for partition, batch := range partitions {
batch.owner.mu.Lock()
if batch.isOwnersFirstBatch() {
if debug {
fmt.Fprintf(b, "%d{0=>%d}, ", partition, len(batch.records))
}
s.cl.finishBatch(batch.recBatch, req.producerID, req.producerEpoch, partition, 0, nil)
} else if debug {
fmt.Fprintf(b, "%d{skipped}, ", partition)
}
batch.owner.mu.Unlock()
}
if debug {
if bytes.HasSuffix(b.Bytes(), []byte(", ")) {
b.Truncate(b.Len() - 2)
}
b.WriteString("], ")
}
}
}
func (s *sink) handleReqResp(br *broker, req *produceRequest, resp kmsg.Response, err error) {
if err != nil {
s.handleReqClientErr(req, err)
return
}
s.firstRespCheck(req.idempotent(), req.version)
s.consecutiveFailures.Store(0)
defer req.metrics.hook(&s.cl.cfg, br) // defer to end so that non-written batches are removed
var b *bytes.Buffer
debug := s.cl.cfg.logger.Level() >= LogLevelDebug
if debug {
b = bytes.NewBuffer(make([]byte, 0, 128))
defer func() {
update := b.String()
update = strings.TrimSuffix(update, ", ")
s.cl.cfg.logger.Log(LogLevelDebug, "produced", "broker", logID(s.nodeID), "to", update)
}()
}
if req.acks == 0 {
s.handleReqRespNoack(b, debug, req)
return
}
var reqRetry seqRecBatches // handled at the end
pr := resp.(*kmsg.ProduceResponse)
for _, rTopic := range pr.Topics {
topic := rTopic.Topic
partitions, ok := req.batches[topic]
if !ok {
s.cl.cfg.logger.Log(LogLevelError, "broker erroneously replied with topic in produce request that we did not produce to", "broker", logID(s.nodeID), "topic", topic)
delete(req.metrics, topic)
continue // should not hit this
}
if debug {
fmt.Fprintf(b, "%s[", topic)
}
tmetrics := req.metrics[topic]
for _, rPartition := range rTopic.Partitions {
partition := rPartition.Partition
batch, ok := partitions[partition]
if !ok {
s.cl.cfg.logger.Log(LogLevelError, "broker erroneously replied with partition in produce request that we did not produce to", "broker", logID(s.nodeID), "topic", rTopic.Topic, "partition", partition)
delete(tmetrics, partition)
continue // should not hit this
}
delete(partitions, partition)
retry, didProduce := s.handleReqRespBatch(
b,
topic,
partition,
batch,
req.producerID,
req.producerEpoch,
rPartition.BaseOffset,
rPartition.ErrorCode,
)
if retry {
reqRetry.addSeqBatch(topic, partition, batch)
}
if !didProduce {
delete(tmetrics, partition)
}
}
if debug {
if bytes.HasSuffix(b.Bytes(), []byte(", ")) {
b.Truncate(b.Len() - 2)
}
b.WriteString("], ")
}
if len(partitions) == 0 {
delete(req.batches, topic)
}
}
if len(req.batches) > 0 {
s.cl.cfg.logger.Log(LogLevelError, "broker did not reply to all topics / partitions in the produce request! reenqueuing missing partitions", "broker", logID(s.nodeID))
s.handleRetryBatches(req.batches, 0, true, false, "broker did not reply to all topics in produce request")
}
if len(reqRetry) > 0 {
s.handleRetryBatches(reqRetry, 0, true, true, "produce request had retry batches")
}
}
func (s *sink) handleReqRespBatch(
b *bytes.Buffer,
topic string,
partition int32,
batch seqRecBatch,
producerID int64,
producerEpoch int16,
baseOffset int64,
errorCode int16,
) (retry, didProduce bool) {
batch.owner.mu.Lock()
defer batch.owner.mu.Unlock()
nrec := len(batch.records)
debug := b != nil
if debug {
fmt.Fprintf(b, "%d{", partition)
}
// We only ever operate on the first batch in a record buf. Batches
// work sequentially; if this is not the first batch then an error
// happened and this later batch is no longer a part of a seq chain.
if !batch.isOwnersFirstBatch() {
if debug {
if err := kerr.ErrorForCode(errorCode); err == nil {
if nrec > 0 {
fmt.Fprintf(b, "skipped@%d=>%d}, ", baseOffset, baseOffset+int64(nrec))
} else {
fmt.Fprintf(b, "skipped@%d}, ", baseOffset)
}
} else {
if nrec > 0 {
fmt.Fprintf(b, "skipped@%d,%d(%s)}, ", baseOffset, nrec, err)
} else {
fmt.Fprintf(b, "skipped@%d(%s)}, ", baseOffset, err)
}
}
}
return false, false
}
// Since we have received a response and we are the first batch, we can
// at this point re-enable failing from load errors.
batch.canFailFromLoadErrs = true
err := kerr.ErrorForCode(errorCode)
failUnknown := batch.owner.checkUnknownFailLimit(err)
switch {
case kerr.IsRetriable(err) &&
!failUnknown &&
err != kerr.CorruptMessage &&
batch.tries < s.cl.cfg.recordRetries:
if debug {
fmt.Fprintf(b, "retrying@%d,%d(%s)}, ", baseOffset, nrec, err)
}
return true, false
case err == kerr.OutOfOrderSequenceNumber,
err == kerr.UnknownProducerID,
err == kerr.InvalidProducerIDMapping,
err == kerr.InvalidProducerEpoch:
// OOOSN always means data loss 1.0+ and is ambiguous prior.
// We assume the worst and only continue if requested.
//
// UnknownProducerID was introduced to allow some form of safe
// handling, but KIP-360 demonstrated that resetting sequence
// numbers is fundamentally unsafe, so we treat it like OOOSN.
//
// InvalidMapping is similar to UnknownProducerID, but occurs
// when the txnal coordinator timed out our transaction.
//
// 2.5
// =====
// 2.5 introduced some behavior to potentially safely reset
// the sequence numbers by bumping an epoch (see KIP-360).
//
// For the idempotent producer, the solution is to fail all
// buffered records and then let the client user reset things
// with the understanding that they cannot guard against
// potential dups / reordering at that point. Realistically,
// that's no better than a config knob that allows the user
// to continue (our stopOnDataLoss flag), so for the idempotent
// producer, if stopOnDataLoss is false, we just continue.
//
// For the transactional producer, we always fail the producerID.
// EndTransaction will trigger recovery if possible.
//
// 2.7
// =====
// InvalidProducerEpoch became retryable in 2.7. Prior, it
// was ambiguous (timeout? fenced?). Now, InvalidProducerEpoch
// is only returned on produce, and then we can recover on other
// txn coordinator requests, which have PRODUCER_FENCED vs
// TRANSACTION_TIMED_OUT.
if s.cl.cfg.txnID != nil || s.cl.cfg.stopOnDataLoss {
s.cl.cfg.logger.Log(LogLevelInfo, "batch errored, failing the producer ID",
"broker", logID(s.nodeID),
"topic", topic,
"partition", partition,
"producer_id", producerID,
"producer_epoch", producerEpoch,
"err", err,
)
s.cl.failProducerID(producerID, producerEpoch, err)
s.cl.finishBatch(batch.recBatch, producerID, producerEpoch, partition, baseOffset, err)
if debug {
fmt.Fprintf(b, "fatal@%d,%d(%s)}, ", baseOffset, nrec, err)
}
return false, false
}
if s.cl.cfg.onDataLoss != nil {
s.cl.cfg.onDataLoss(topic, partition)
}
// For OOOSN, and UnknownProducerID
//
// The only recovery is to fail the producer ID, which ensures
// that all batches reset sequence numbers and use a new producer
// ID on the next batch.
//
// For InvalidProducerIDMapping && InvalidProducerEpoch,
//
// We should not be here, since this error occurs in the
// context of transactions, which are caught above.
s.cl.cfg.logger.Log(LogLevelInfo, fmt.Sprintf("batch errored with %s, failing the producer ID and resetting all sequence numbers", err.(*kerr.Error).Message),
"broker", logID(s.nodeID),
"topic", topic,
"partition", partition,
"producer_id", producerID,
"producer_epoch", producerEpoch,
"err", err,
)
// After we fail here, any new produce (even new ones
// happening concurrent with this function) will load
// a new epoch-bumped producer ID and all first-batches
// will reset sequence numbers appropriately.
s.cl.failProducerID(producerID, producerEpoch, errReloadProducerID)
if debug {
fmt.Fprintf(b, "resetting@%d,%d(%s)}, ", baseOffset, nrec, err)
}
return true, false
case err == kerr.DuplicateSequenceNumber: // ignorable, but we should not get
s.cl.cfg.logger.Log(LogLevelInfo, "received unexpected duplicate sequence number, ignoring and treating batch as successful",
"broker", logID(s.nodeID),
"topic", topic,
"partition", partition,
)
err = nil
fallthrough
default:
if err != nil {
s.cl.cfg.logger.Log(LogLevelInfo, "batch in a produce request failed",
"broker", logID(s.nodeID),
"topic", topic,
"partition", partition,
"err", err,
"err_is_retryable", kerr.IsRetriable(err),
"max_retries_reached", !failUnknown && batch.tries >= s.cl.cfg.recordRetries,
)
batch.owner.okOnSink = false
} else {
batch.owner.okOnSink = true
}
s.cl.finishBatch(batch.recBatch, producerID, producerEpoch, partition, baseOffset, err)
didProduce = err == nil
if debug {
if err != nil {
fmt.Fprintf(b, "err@%d,%d(%s)}, ", baseOffset, nrec, err)
} else {
fmt.Fprintf(b, "%d=>%d}, ", baseOffset, baseOffset+int64(nrec))
}
}
}
return false, didProduce // no retry
}
// finishBatch removes a batch from its owning record buffer and finishes all
// records in the batch.
//
// This is safe even if the owning recBuf migrated sinks, since we are
// finishing based off the status of an inflight req from the original sink.
func (cl *Client) finishBatch(batch *recBatch, producerID int64, producerEpoch int16, partition int32, baseOffset int64, err error) {
recBuf := batch.owner
if err != nil {
// We know that Kafka replied this batch is a failure. We can
// fail this batch and all batches in this partition.
// This will keep sequence numbers correct.
recBuf.failAllRecords(err)
return
}
// We know the batch made it to Kafka successfully without error.
// We remove this batch and finish all records appropriately.
finished := len(batch.records)
recBuf.batch0Seq = incrementSequence(recBuf.batch0Seq, int32(finished))
recBuf.buffered.Add(-int64(finished))
recBuf.batches[0] = nil
recBuf.batches = recBuf.batches[1:]
recBuf.batchDrainIdx--
batch.mu.Lock()
records, attrs := batch.records, batch.attrs
batch.records = nil
batch.mu.Unlock()
cl.producer.promiseBatch(batchPromise{
baseOffset: baseOffset,
pid: producerID,
epoch: producerEpoch,
// A recBuf.attrs is updated when appending to be written. For
// v0 && v1 produce requests, we set bit 8 in the attrs
// corresponding to our own RecordAttr's bit 8 being no
// timestamp type. Thus, we can directly convert the batch
// attrs to our own RecordAttrs.
attrs: RecordAttrs{uint8(attrs)},
partition: partition,
recs: records,
})
}
// handleRetryBatches sets any first-buf-batch to failing and triggers a
// metadata that will eventually clear the failing state and re-drain.
//
// If idempotency is disabled, if a batch is timed out or hit the retry limit,
// we fail it and anything after it.
func (s *sink) handleRetryBatches(
retry seqRecBatches,
backoffSeq uint32,
updateMeta bool, // if we should maybe update the metadata
canFail bool, // if records can fail if they are at limits
why string,
) {
logger := s.cl.cfg.logger
debug := logger.Level() >= LogLevelDebug
var needsMetaUpdate bool
var shouldBackoff bool
retry.eachOwnerLocked(func(batch seqRecBatch) {
if !batch.isOwnersFirstBatch() {
if debug {
logger.Log(LogLevelDebug, "retry batch is not the first batch in the owner, skipping result",
"topic", batch.owner.topic,
"partition", batch.owner.partition,
)
}
return
}
if canFail || s.cl.cfg.disableIdempotency {
if err := batch.maybeFailErr(&s.cl.cfg); err != nil {
batch.owner.failAllRecords(err)
return
}
}
batch.owner.resetBatchDrainIdx()
// If our first batch (seq == 0) fails with unknown topic, we
// retry immediately. Kafka can reply with valid metadata
// immediately after a topic was created, before the leaders
// actually know they are leader.
unknownAndFirstBatch := batch.owner.unknownFailures == 1 && batch.owner.seq == 0
if unknownAndFirstBatch {
shouldBackoff = true
return
}
if updateMeta {
batch.owner.failing = true
needsMetaUpdate = true
}
})
if debug {
logger.Log(LogLevelDebug, "retry batches processed",
"wanted_metadata_update", updateMeta,
"triggering_metadata_update", needsMetaUpdate,
"should_backoff", shouldBackoff,
)
}
// If we do want to metadata update, we only do so if any batch was the
// first batch in its buf / not concurrently failed.
if needsMetaUpdate {
s.cl.triggerUpdateMetadata(true, why)
return
}
// We could not need a metadata update for two reasons:
//
// * our request died when being issued
//
// * we would update metadata, but what failed was the first batch
// produced and the error was unknown topic / partition.
//
// In either of these cases, we should backoff a little bit to avoid
// spin looping.
//
// If neither of these cases are true, then we entered wanting a
// metadata update, but the batches either were not the first batch, or
// the batches were concurrently failed.
if shouldBackoff || !updateMeta {
s.maybeTriggerBackoff(backoffSeq)
s.maybeDrain()
}
}
// addRecBuf adds a new record buffer to be drained to a sink and clears the
// buffer's failing state.
func (s *sink) addRecBuf(add *recBuf) {
s.recBufsMu.Lock()
add.recBufsIdx = len(s.recBufs)
s.recBufs = append(s.recBufs, add)
s.recBufsMu.Unlock()
add.clearFailing()
}
// removeRecBuf removes a record buffer from a sink.
func (s *sink) removeRecBuf(rm *recBuf) {
s.recBufsMu.Lock()
defer s.recBufsMu.Unlock()
if rm.recBufsIdx != len(s.recBufs)-1 {