-
-
Notifications
You must be signed in to change notification settings - Fork 196
/
source.go
2027 lines (1796 loc) · 57.4 KB
/
source.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"
"encoding/binary"
"fmt"
"hash/crc32"
"sort"
"sync"
"time"
"github.com/twmb/franz-go/pkg/kbin"
"github.com/twmb/franz-go/pkg/kerr"
"github.com/twmb/franz-go/pkg/kmsg"
)
type readerFrom interface {
ReadFrom([]byte) error
}
// A source consumes from an individual broker.
//
// As long as there is at least one active cursor, a source aims to have *one*
// buffered fetch at all times. As soon as the fetch is taken, a source issues
// another fetch in the background.
type source struct {
cl *Client // our owning client, for cfg, metadata triggering, context, etc.
nodeID int32 // the node ID of the broker this sink belongs to
// Tracks how many _failed_ fetch requests we have in a row (unable to
// receive a response). Any response, even responses with an ErrorCode
// set, are successful. This field is used for backoff purposes.
consecutiveFailures int
fetchState workLoop
sem chan struct{} // closed when fetchable, recreated when a buffered fetch exists
buffered bufferedFetch // contains a fetch the source has buffered for polling
session fetchSession // supports fetch sessions as per KIP-227
cursorsMu sync.Mutex
cursors []*cursor // contains all partitions being consumed on this source
cursorsStart int // incremented every fetch req to ensure all partitions are fetched
}
func (cl *Client) newSource(nodeID int32) *source {
s := &source{
cl: cl,
nodeID: nodeID,
sem: make(chan struct{}),
}
if cl.cfg.disableFetchSessions {
s.session.kill()
}
close(s.sem)
return s
}
func (s *source) addCursor(add *cursor) {
s.cursorsMu.Lock()
add.cursorsIdx = len(s.cursors)
s.cursors = append(s.cursors, add)
s.cursorsMu.Unlock()
// Adding a new cursor may allow a new partition to be fetched.
// We do not need to cancel any current fetch nor kill the session,
// since adding a cursor is non-destructive to work in progress.
// If the session is currently stopped, this is a no-op.
s.maybeConsume()
}
// Removes a cursor from the source.
//
// The caller should do this with a stopped session if necessary, which
// should clear any buffered fetch and reset the source's session.
func (s *source) removeCursor(rm *cursor) {
s.cursorsMu.Lock()
defer s.cursorsMu.Unlock()
if rm.cursorsIdx != len(s.cursors)-1 {
s.cursors[rm.cursorsIdx], s.cursors[len(s.cursors)-1] = s.cursors[len(s.cursors)-1], nil
s.cursors[rm.cursorsIdx].cursorsIdx = rm.cursorsIdx
} else {
s.cursors[rm.cursorsIdx] = nil // do not let the memory hang around
}
s.cursors = s.cursors[:len(s.cursors)-1]
if s.cursorsStart == len(s.cursors) {
s.cursorsStart = 0
}
}
// cursor is where we are consuming from for an individual partition.
type cursor struct {
topic string
topicID [16]byte
partition int32
unknownIDFails atomicI32
keepControl bool // whether to keep control records
cursorsIdx int // updated under source mutex
// The source we are currently on. This is modified in two scenarios:
//
// * by metadata when the consumer session is completely stopped
//
// * by a fetch when handling a fetch response that returned preferred
// replicas
//
// This is additionally read within a session when cursor is
// transitioning from used to usable.
source *source
// useState is an atomic that has two states: unusable and usable. A
// cursor can be used in a fetch request if it is in the usable state.
// Once used, the cursor is unusable, and will be set back to usable
// one the request lifecycle is complete (a usable fetch response, or
// once listing offsets or loading epochs completes).
//
// A cursor can be set back to unusable when sources are stopped. This
// can be done if a group loses a partition, for example.
//
// The used state is exclusively updated by either building a fetch
// request or when the source is stopped.
useState atomicBool
topicPartitionData // updated in metadata when session is stopped
// cursorOffset is our epoch/offset that we are consuming. When a fetch
// request is issued, we "freeze" a view of the offset and of the
// leader epoch (see cursorOffsetNext for why the leader epoch). When a
// buffered fetch is taken, we update the cursor.
cursorOffset
}
// cursorOffset tracks offsets/epochs for a cursor.
type cursorOffset struct {
// What the cursor is at: we request this offset next.
offset int64
// The epoch of the last record we consumed. Also used for KIP-320, if
// we are fenced or we have an offset out of range error, we go into
// the OffsetForLeaderEpoch recovery. The last consumed epoch tells the
// broker which offset we want: either (a) the next offset if the last
// consumed epoch is the current epoch, or (b) the offset of the first
// record in the next epoch. This allows for exact offset resetting and
// data loss detection.
//
// See kmsg.OffsetForLeaderEpochResponseTopicPartition for more
// details.
lastConsumedEpoch int32
// The current high watermark of the partition. Uninitialized (0) means
// we do not know the HWM, or there is no lag.
hwm int64
}
// use, for fetch requests, freezes a view of the cursorOffset.
func (c *cursor) use() *cursorOffsetNext {
// A source using a cursor has exclusive access to the use field by
// virtue of that source building a request during a live session,
// or by virtue of the session being stopped.
c.useState.Store(false)
return &cursorOffsetNext{
cursorOffset: c.cursorOffset,
from: c,
currentLeaderEpoch: c.leaderEpoch,
}
}
// unset transitions a cursor to an unusable state when the cursor is no longer
// to be consumed. This is called exclusively after sources are stopped.
// This also unsets the cursor offset, which is assumed to be unused now.
func (c *cursor) unset() {
c.useState.Store(false)
c.setOffset(cursorOffset{
offset: -1,
lastConsumedEpoch: -1,
hwm: 0,
})
}
// usable returns whether a cursor can be used for building a fetch request.
func (c *cursor) usable() bool {
return c.useState.Load()
}
// allowUsable allows a cursor to be fetched, and is called either in assigning
// offsets, or when a buffered fetch is taken or discarded, or when listing /
// epoch loading finishes.
func (c *cursor) allowUsable() {
c.useState.Swap(true)
c.source.maybeConsume()
}
// setOffset sets the cursors offset which will be used the next time a fetch
// request is built. This function is called under the source mutex while the
// source is stopped, and the caller is responsible for calling maybeConsume
// after.
func (c *cursor) setOffset(o cursorOffset) {
c.cursorOffset = o
}
// cursorOffsetNext is updated while processing a fetch response.
//
// When a buffered fetch is taken, we update a cursor with the final values in
// the modified cursor offset.
type cursorOffsetNext struct {
cursorOffset
from *cursor
// The leader epoch at the time we took this cursor offset snapshot. We
// need to copy this rather than accessing it through `from` because a
// fetch request can be canceled while it is being written (and reading
// the epoch).
//
// The leader field itself is only read within the context of a session
// while the session is alive, thus it needs no such guard.
//
// Basically, any field read in AppendTo needs to be copied into
// cursorOffsetNext.
currentLeaderEpoch int32
}
type cursorOffsetPreferred struct {
cursorOffsetNext
preferredReplica int32
}
// Moves a cursor from one source to another. This is done while handling
// a fetch response, which means within the context of a live session.
func (p *cursorOffsetPreferred) move() {
c := p.from
defer c.allowUsable()
// Before we migrate the cursor, we check if the destination source
// exists. If not, we do not migrate and instead force a metadata.
c.source.cl.sinksAndSourcesMu.Lock()
sns, exists := c.source.cl.sinksAndSources[p.preferredReplica]
c.source.cl.sinksAndSourcesMu.Unlock()
if !exists {
c.source.cl.triggerUpdateMetadataNow("cursor moving to a different broker that is not yet known")
return
}
// This remove clears the source's session and buffered fetch, although
// we will not have a buffered fetch since moving replicas is called
// before buffering a fetch.
c.source.removeCursor(c)
c.source = sns.source
c.source.addCursor(c)
}
type cursorPreferreds []cursorOffsetPreferred
func (cs cursorPreferreds) eachPreferred(fn func(cursorOffsetPreferred)) {
for _, c := range cs {
fn(c)
}
}
type usedOffsets map[string]map[int32]*cursorOffsetNext
func (os usedOffsets) eachOffset(fn func(*cursorOffsetNext)) {
for _, ps := range os {
for _, o := range ps {
fn(o)
}
}
}
func (os usedOffsets) finishUsingAllWithSet() {
os.eachOffset(func(o *cursorOffsetNext) { o.from.setOffset(o.cursorOffset); o.from.allowUsable() })
}
func (os usedOffsets) finishUsingAll() {
os.eachOffset(func(o *cursorOffsetNext) { o.from.allowUsable() })
}
// bufferedFetch is a fetch response waiting to be consumed by the client.
type bufferedFetch struct {
fetch Fetch
doneFetch chan<- struct{} // when unbuffered, we send down this
usedOffsets usedOffsets // what the offsets will be next if this fetch is used
}
func (s *source) hook(f *Fetch, buffered, polled bool) {
s.cl.cfg.hooks.each(func(h Hook) {
if buffered {
h, ok := h.(HookFetchRecordBuffered)
if !ok {
return
}
for i := range f.Topics {
t := &f.Topics[i]
for j := range t.Partitions {
p := &t.Partitions[j]
for _, r := range p.Records {
h.OnFetchRecordBuffered(r)
}
}
}
} else {
h, ok := h.(HookFetchRecordUnbuffered)
if !ok {
return
}
for i := range f.Topics {
t := &f.Topics[i]
for j := range t.Partitions {
p := &t.Partitions[j]
for _, r := range p.Records {
h.OnFetchRecordUnbuffered(r, polled)
}
}
}
}
})
var nrecs int
for i := range f.Topics {
t := &f.Topics[i]
for j := range t.Partitions {
nrecs += len(t.Partitions[j].Records)
}
}
if buffered {
s.cl.consumer.bufferedRecords.Add(int64(nrecs))
} else {
s.cl.consumer.bufferedRecords.Add(-int64(nrecs))
}
}
// takeBuffered drains a buffered fetch and updates offsets.
func (s *source) takeBuffered() Fetch {
return s.takeBufferedFn(true, usedOffsets.finishUsingAllWithSet)
}
func (s *source) discardBuffered() {
s.takeBufferedFn(false, usedOffsets.finishUsingAll)
}
// takeNBuffered takes a limited amount of records from a buffered fetch,
// updating offsets in each partition per records taken.
//
// This only allows a new fetch once every buffered record has been taken.
//
// This returns the number of records taken and whether the source has been
// completely drained.
func (s *source) takeNBuffered(n int) (Fetch, int, bool) {
var r Fetch
var taken int
b := &s.buffered
bf := &b.fetch
for len(bf.Topics) > 0 && n > 0 {
t := &bf.Topics[0]
r.Topics = append(r.Topics, *t)
rt := &r.Topics[len(r.Topics)-1]
rt.Partitions = nil
tCursors := b.usedOffsets[t.Topic]
for len(t.Partitions) > 0 && n > 0 {
p := &t.Partitions[0]
rt.Partitions = append(rt.Partitions, *p)
rp := &rt.Partitions[len(rt.Partitions)-1]
take := n
if take > len(p.Records) {
take = len(p.Records)
}
rp.Records = p.Records[:take:take]
p.Records = p.Records[take:]
n -= take
taken += take
pCursor := tCursors[p.Partition]
if len(p.Records) == 0 {
t.Partitions = t.Partitions[1:]
pCursor.from.setOffset(pCursor.cursorOffset)
pCursor.from.allowUsable()
delete(tCursors, p.Partition)
if len(tCursors) == 0 {
delete(b.usedOffsets, t.Topic)
}
break
}
lastReturnedRecord := rp.Records[len(rp.Records)-1]
pCursor.from.setOffset(cursorOffset{
offset: lastReturnedRecord.Offset + 1,
lastConsumedEpoch: lastReturnedRecord.LeaderEpoch,
hwm: p.HighWatermark,
})
}
if len(t.Partitions) == 0 {
bf.Topics = bf.Topics[1:]
}
}
s.hook(&r, false, true) // unbuffered, polled
drained := len(bf.Topics) == 0
if drained {
s.takeBuffered()
}
return r, taken, drained
}
func (s *source) takeBufferedFn(polled bool, offsetFn func(usedOffsets)) Fetch {
r := s.buffered
s.buffered = bufferedFetch{}
offsetFn(r.usedOffsets)
r.doneFetch <- struct{}{}
close(s.sem)
s.hook(&r.fetch, false, polled) // unbuffered, potentially polled
return r.fetch
}
// createReq actually creates a fetch request.
func (s *source) createReq() *fetchRequest {
req := &fetchRequest{
maxWait: s.cl.cfg.maxWait,
minBytes: s.cl.cfg.minBytes,
maxBytes: s.cl.cfg.maxBytes.load(),
maxPartBytes: s.cl.cfg.maxPartBytes.load(),
rack: s.cl.cfg.rack,
isolationLevel: s.cl.cfg.isolationLevel,
preferLagFn: s.cl.cfg.preferLagFn,
// We copy a view of the session for the request, which allows
// modify source while the request may be reading its copy.
session: s.session,
}
paused := s.cl.consumer.loadPaused()
s.cursorsMu.Lock()
defer s.cursorsMu.Unlock()
cursorIdx := s.cursorsStart
for i := 0; i < len(s.cursors); i++ {
c := s.cursors[cursorIdx]
cursorIdx = (cursorIdx + 1) % len(s.cursors)
if !c.usable() || paused.has(c.topic, c.partition) {
continue
}
req.addCursor(c)
}
// We could have lost our only record buffer just before we grabbed the
// source lock above.
if len(s.cursors) > 0 {
s.cursorsStart = (s.cursorsStart + 1) % len(s.cursors)
}
return req
}
func (s *source) maybeConsume() {
if s.fetchState.maybeBegin() {
go s.loopFetch()
}
}
func (s *source) loopFetch() {
consumer := &s.cl.consumer
session := consumer.loadSession()
if session == noConsumerSession {
s.fetchState.hardFinish()
return
}
session.incWorker()
defer session.decWorker()
// After our add, check quickly **without** another select case to
// determine if this context was truly canceled. Any other select that
// has another select case could theoretically race with the other case
// also being selected.
select {
case <-session.ctx.Done():
s.fetchState.hardFinish()
return
default:
}
// We receive on canFetch when we can fetch, and we send back when we
// are done fetching.
canFetch := make(chan chan struct{}, 1)
again := true
for again {
select {
case <-session.ctx.Done():
s.fetchState.hardFinish()
return
case <-s.sem:
}
select {
case <-session.ctx.Done():
s.fetchState.hardFinish()
return
case session.desireFetch() <- canFetch:
}
select {
case <-session.ctx.Done():
session.cancelFetchCh <- canFetch
s.fetchState.hardFinish()
return
case doneFetch := <-canFetch:
again = s.fetchState.maybeFinish(s.fetch(session, doneFetch))
}
}
}
func (s *source) killSessionOnClose(ctx context.Context) {
br, err := s.cl.brokerOrErr(nil, s.nodeID, errUnknownBroker)
if err != nil {
return
}
s.session.kill()
req := &fetchRequest{
maxWait: 1,
minBytes: 1,
maxBytes: 1,
maxPartBytes: 1,
rack: s.cl.cfg.rack,
isolationLevel: s.cl.cfg.isolationLevel,
session: s.session,
}
ch := make(chan struct{})
br.do(ctx, req, func(kmsg.Response, error) { close(ch) })
<-ch
}
// fetch is the main logic center of fetching messages.
//
// This is a long function, made much longer by winded documentation, that
// contains a lot of the side effects of fetching and updating. The function
// consists of two main bulks of logic:
//
// - First, issue a request that can be killed if the source needs to be
// stopped. Processing the response modifies no state on the source.
//
// - Second, we keep the fetch response and update everything relevant
// (session, trigger some list or epoch updates, buffer the fetch).
//
// One small part between the first and second step is to update preferred
// replicas. We always keep the preferred replicas from the fetch response
// *even if* the source needs to be stopped. The knowledge of which preferred
// replica to use would not be out of date even if the consumer session is
// changing.
func (s *source) fetch(consumerSession *consumerSession, doneFetch chan<- struct{}) (fetched bool) {
req := s.createReq()
// For all returns, if we do not buffer our fetch, then we want to
// ensure our used offsets are usable again.
var (
alreadySentToDoneFetch bool
setOffsets bool
buffered bool
)
defer func() {
if !buffered {
if req.numOffsets > 0 {
if setOffsets {
req.usedOffsets.finishUsingAllWithSet()
} else {
req.usedOffsets.finishUsingAll()
}
}
if !alreadySentToDoneFetch {
doneFetch <- struct{}{}
}
}
}()
if req.numOffsets == 0 { // cursors could have been set unusable
return
}
// If our fetch is killed, we want to cancel waiting for the response.
var (
kresp kmsg.Response
requested = make(chan struct{})
ctx, cancel = context.WithCancel(consumerSession.ctx)
)
defer cancel()
br, err := s.cl.brokerOrErr(ctx, s.nodeID, errUnknownBroker)
if err != nil {
close(requested)
} else {
br.do(ctx, req, func(k kmsg.Response, e error) {
kresp, err = k, e
close(requested)
})
}
select {
case <-requested:
fetched = true
case <-ctx.Done():
return
}
var didBackoff bool
backoff := func() {
// We preemptively allow more fetches (since we are not buffering)
// and reset our session because of the error (who knows if kafka
// processed the request but the client failed to receive it).
doneFetch <- struct{}{}
alreadySentToDoneFetch = true
s.session.reset()
didBackoff = true
s.cl.triggerUpdateMetadata(false, "opportunistic load during source backoff") // as good a time as any
s.consecutiveFailures++
after := time.NewTimer(s.cl.cfg.retryBackoff(s.consecutiveFailures))
defer after.Stop()
select {
case <-after.C:
case <-ctx.Done():
}
}
defer func() {
if !didBackoff {
s.consecutiveFailures = 0
}
}()
// If we had an error, we backoff. Killing a fetch quits the backoff,
// but that is fine; we may just re-request too early and fall into
// another backoff.
if err != nil {
backoff()
return
}
resp := kresp.(*kmsg.FetchResponse)
var (
fetch Fetch
reloadOffsets listOrEpochLoads
preferreds cursorPreferreds
allErrsStripped bool
updateMeta bool
updateWhy string
handled = make(chan struct{})
)
// Theoretically, handleReqResp could take a bit of CPU time due to
// decompressing and processing the response. We do this in a goroutine
// to allow the session to be canceled at any moment.
//
// Processing the response only needs the source's nodeID and client.
go func() {
defer close(handled)
fetch, reloadOffsets, preferreds, allErrsStripped, updateMeta, updateWhy = s.handleReqResp(br, req, resp)
}()
select {
case <-handled:
case <-ctx.Done():
return
}
// The logic below here should be relatively quick.
deleteReqUsedOffset := func(topic string, partition int32) {
t := req.usedOffsets[topic]
delete(t, partition)
if len(t) == 0 {
delete(req.usedOffsets, topic)
}
}
// Before updating the source, we move all cursors that have new
// preferred replicas and remove them from being tracked in our req
// offsets. We also remove the reload offsets from our req offsets.
//
// These two removals transition responsibility for finishing using the
// cursor from the request's used offsets to the new source or the
// reloading.
preferreds.eachPreferred(func(c cursorOffsetPreferred) {
c.move()
deleteReqUsedOffset(c.from.topic, c.from.partition)
})
reloadOffsets.each(deleteReqUsedOffset)
// The session on the request was updated; we keep those updates.
s.session = req.session
// handleReqResp only parses the body of the response, not the top
// level error code.
//
// The top level error code is related to fetch sessions only, and if
// there was an error, the body was empty (so processing is basically a
// no-op). We process the fetch session error now.
switch err := kerr.ErrorForCode(resp.ErrorCode); err {
case kerr.FetchSessionIDNotFound:
if s.session.epoch == 0 {
// If the epoch was zero, the broker did not even
// establish a session for us (and thus is maxed on
// sessions). We stop trying.
s.cl.cfg.logger.Log(LogLevelInfo, "session failed with SessionIDNotFound while trying to establish a session; broker likely maxed on sessions; continuing on without using sessions", "broker", logID(s.nodeID))
s.session.kill()
} else {
s.cl.cfg.logger.Log(LogLevelInfo, "received SessionIDNotFound from our in use session, our session was likely evicted; resetting session", "broker", logID(s.nodeID))
s.session.reset()
}
return
case kerr.InvalidFetchSessionEpoch:
s.cl.cfg.logger.Log(LogLevelInfo, "resetting fetch session", "broker", logID(s.nodeID), "err", err)
s.session.reset()
return
case kerr.FetchSessionTopicIDError, kerr.InconsistentTopicID:
s.cl.cfg.logger.Log(LogLevelInfo, "topic id issues, resetting session and updating metadata", "broker", logID(s.nodeID), "err", err)
s.session.reset()
s.cl.triggerUpdateMetadataNow("topic id issues")
return
}
// At this point, we have successfully processed the response. Even if
// the response contains no records, we want to keep any offset
// advancements (we could have consumed only control records, we must
// advance past them).
setOffsets = true
if resp.Version < 7 || resp.SessionID <= 0 {
// If the version is less than 7, we cannot use fetch sessions,
// so we kill them on the first response.
s.session.kill()
} else {
s.session.bumpEpoch(resp.SessionID)
}
if updateMeta && !reloadOffsets.loadWithSessionNow(consumerSession, updateWhy) {
s.cl.triggerUpdateMetadataNow(updateWhy)
}
if fetch.hasErrorsOrRecords() {
buffered = true
s.buffered = bufferedFetch{
fetch: fetch,
doneFetch: doneFetch,
usedOffsets: req.usedOffsets,
}
s.sem = make(chan struct{})
s.hook(&fetch, true, false) // buffered, not polled
s.cl.consumer.addSourceReadyForDraining(s)
} else if allErrsStripped {
// If we stripped all errors from the response, we are likely
// fetching from topics that were deleted. We want to back off
// a bit rather than spin-loop immediately re-requesting
// deleted topics.
backoff()
}
return
}
// Parses a fetch response into a Fetch, offsets to reload, and whether
// metadata needs updating.
//
// This only uses a source's broker and client, and thus does not need
// the source mutex.
//
// This function, and everything it calls, is side effect free.
func (s *source) handleReqResp(br *broker, req *fetchRequest, resp *kmsg.FetchResponse) (
f Fetch,
reloadOffsets listOrEpochLoads,
preferreds cursorPreferreds,
allErrsStripped bool,
updateMeta bool,
why string,
) {
f = Fetch{Topics: make([]FetchTopic, 0, len(resp.Topics))}
var (
updateWhy multiUpdateWhy
numErrsStripped int
kip320 = s.cl.supportsOffsetForLeaderEpoch()
)
for _, rt := range resp.Topics {
topic := rt.Topic
// v13 only uses topic IDs, so we have to map the response
// uuid's to our string topics.
if resp.Version >= 13 {
topic = req.id2topic[rt.TopicID]
}
// We always include all cursors on this source in the fetch;
// we should not receive any topics or partitions we do not
// expect.
topicOffsets, ok := req.usedOffsets[topic]
if !ok {
s.cl.cfg.logger.Log(LogLevelWarn, "broker returned topic from fetch that we did not ask for",
"broker", logID(s.nodeID),
"topic", topic,
)
continue
}
fetchTopic := FetchTopic{
Topic: topic,
Partitions: make([]FetchPartition, 0, len(rt.Partitions)),
}
for i := range rt.Partitions {
rp := &rt.Partitions[i]
partition := rp.Partition
partOffset, ok := topicOffsets[partition]
if !ok {
s.cl.cfg.logger.Log(LogLevelWarn, "broker returned partition from fetch that we did not ask for",
"broker", logID(s.nodeID),
"topic", topic,
"partition", partition,
)
continue
}
// If we are fetching from the replica already, Kafka replies with a -1
// preferred read replica. If Kafka replies with a preferred replica,
// it sends no records.
if preferred := rp.PreferredReadReplica; resp.Version >= 11 && preferred >= 0 {
preferreds = append(preferreds, cursorOffsetPreferred{
*partOffset,
preferred,
})
continue
}
fp := partOffset.processRespPartition(br, rp, s.cl.decompressor, s.cl.cfg.hooks)
if fp.Err != nil {
updateMeta = true
updateWhy.add(topic, partition, fp.Err)
}
// We only keep the partition if it has no error, or an
// error we do not internally retry.
var keep bool
switch fp.Err {
default:
if kerr.IsRetriable(fp.Err) {
// UnknownLeaderEpoch: our meta is newer than the broker we fetched from
// OffsetNotAvailable: fetched from out of sync replica or a behind in-sync one (KIP-392 case 1 and case 2)
// UnknownTopicID: kafka has not synced the state on all brokers
// And other standard retryable errors.
numErrsStripped++
} else {
// - bad auth
// - unsupported compression
// - unsupported message version
// - unknown error
// - or, no error
keep = true
}
case nil:
partOffset.from.unknownIDFails.Store(0)
keep = true
case kerr.UnknownTopicID:
// We need to keep UnknownTopicID even though it is
// retryable, because encountering this error means
// the topic has been recreated and we will never
// consume the topic again anymore. This is an error
// worth bubbling up.
//
// FUN FACT: Kafka will actually return this error
// for a brief window immediately after creating a
// topic for the first time, meaning the controller
// has not yet propagated to the leader that it is
// the leader of a new partition. We need to ignore
// this error for a _litttttlllleee bit_.
if fails := partOffset.from.unknownIDFails.Add(1); fails > 5 {
partOffset.from.unknownIDFails.Add(-1)
keep = true
} else {
numErrsStripped++
}
case kerr.OffsetOutOfRange:
// If we are out of range, we reset to what we can.
// With Kafka >= 2.1, we should only get offset out
// of range if we fetch before the start, but a user
// could start past the end and want to reset to
// the end. We respect that.
//
// KIP-392 (case 3) specifies that if we are consuming
// from a follower, then if our offset request is before
// the low watermark, we list offsets from the follower.
//
// KIP-392 (case 4) specifies that if we are consuming
// a follower and our request is larger than the high
// watermark, then we should first check for truncation
// from the leader and then if we still get out of
// range, reset with list offsets.
//
// It further goes on to say that "out of range errors
// due to ISR propagation delays should be extremely
// rare". Rather than falling back to listing offsets,
// we stay in a cycle of validating the leader epoch
// until the follower has caught up.
//
// In all cases except case 4, we also have to check if
// no reset offset was configured. If so, we ignore
// trying to reset and instead keep our failed partition.
addList := func(replica int32) {
if s.cl.cfg.resetOffset.noReset {
keep = true
} else {
reloadOffsets.addLoad(topic, partition, loadTypeList, offsetLoad{
replica: replica,
Offset: s.cl.cfg.resetOffset,
})
}
}
switch {
case s.nodeID == partOffset.from.leader: // non KIP-392 case
addList(-1)
case partOffset.offset < fp.LogStartOffset: // KIP-392 case 3
addList(s.nodeID)
default: // partOffset.offset > fp.HighWatermark, KIP-392 case 4
if kip320 {
reloadOffsets.addLoad(topic, partition, loadTypeEpoch, offsetLoad{
replica: -1,
Offset: Offset{
at: partOffset.offset,
epoch: partOffset.lastConsumedEpoch,
},
})
} else {
// If the broker does not support offset for leader epoch but
// does support follower fetching for some reason, we have to
// fallback to listing.
addList(-1)
}
}
case kerr.FencedLeaderEpoch:
// With fenced leader epoch, we notify an error only
// if necessary after we find out if loss occurred.
// If we have consumed nothing, then we got unlucky
// by being fenced right after we grabbed metadata.
// We just refresh metadata and try again.
//
// It would be odd for a broker to reply we are fenced
// but not support offset for leader epoch, so we do
// not check KIP-320 support here.
if partOffset.lastConsumedEpoch >= 0 {
reloadOffsets.addLoad(topic, partition, loadTypeEpoch, offsetLoad{
replica: -1,
Offset: Offset{
at: partOffset.offset,
epoch: partOffset.lastConsumedEpoch,
},
})
}
}
if keep {
fetchTopic.Partitions = append(fetchTopic.Partitions, fp)
}
}
if len(fetchTopic.Partitions) > 0 {
f.Topics = append(f.Topics, fetchTopic)
}
}
return f, reloadOffsets, preferreds, req.numOffsets == numErrsStripped, updateMeta, updateWhy.reason("fetch had inner topic errors")
}
// processRespPartition processes all records in all potentially compressed
// batches (or message sets).
func (o *cursorOffsetNext) processRespPartition(br *broker, rp *kmsg.FetchResponseTopicPartition, decompressor *decompressor, hooks hooks) FetchPartition {