forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
reader.go
2081 lines (1754 loc) · 57.4 KB
/
reader.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 kafka
import (
"bufio"
"bytes"
"context"
"errors"
"fmt"
"io"
"log"
"math"
"sort"
"strconv"
"sync"
"sync/atomic"
"time"
)
const (
LastOffset int64 = -1 // The most recent offset available for a partition.
FirstOffset = -2 // The least recent offset available for a partition.
)
const (
// defaultCommitRetries holds the number commit attempts to make
// before giving up
defaultCommitRetries = 3
)
var (
errOnlyAvailableWithGroup = errors.New("unavailable when GroupID is not set")
errNotAvailableWithGroup = errors.New("unavailable when GroupID is set")
)
const (
// defaultProtocolType holds the default protocol type documented in the
// kafka protocol
//
// See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipAPI
defaultProtocolType = "consumer"
// defaultHeartbeatInterval contains the default time between heartbeats. If
// the coordinator does not receive a heartbeat within the session timeout interval,
// the consumer will be considered dead and the coordinator will rebalance the
// group.
//
// As a rule, the heartbeat interval should be no greater than 1/3 the session timeout
defaultHeartbeatInterval = 3 * time.Second
// defaultSessionTimeout contains the default interval the coordinator will wait
// for a heartbeat before marking a consumer as dead
defaultSessionTimeout = 30 * time.Second
// defaultRebalanceTimeout contains the amount of time the coordinator will wait
// for consumers to issue a join group once a rebalance has been requested
defaultRebalanceTimeout = 30 * time.Second
// defaultRetentionTime holds the length of time a the consumer group will be
// saved by kafka
defaultRetentionTime = time.Hour * 24
// defaultPartitionWatchTime contains the amount of time the kafka-go will wait to
// query the brokers looking for partition changes.
defaultPartitionWatchTime = 5 * time.Second
// defaultReadBackoffMax/Min sets the boundaries for how long the reader wait before
// polling for new messages
defaultReadBackoffMin = 100 * time.Millisecond
defaultReadBackoffMax = 1 * time.Second
)
// Reader provides a high-level API for consuming messages from kafka.
//
// A Reader automatically manages reconnections to a kafka server, and
// blocking methods have context support for asynchronous cancellations.
type Reader struct {
// immutable fields of the reader
config ReaderConfig
// communication channels between the parent reader and its subreaders
msgs chan readerMessage
// mutable fields of the reader (synchronized on the mutex)
mutex sync.Mutex
join sync.WaitGroup
cancel context.CancelFunc
stop context.CancelFunc
done chan struct{}
commits chan commitRequest
version int64 // version holds the generation of the spawned readers
offset int64
lag int64
closed bool
address string // address of group coordinator
generationID int32 // generationID of group
memberID string // memberID of group
// offsetStash should only be managed by the commitLoopInterval. We store
// it here so that it survives rebalances
offsetStash offsetStash
// reader stats are all made of atomic values, no need for synchronization.
once uint32
stctx context.Context
// reader stats are all made of atomic values, no need for synchronization.
// Use a pointer to ensure 64-bit alignment of the values.
stats *readerStats
}
// useConsumerGroup indicates whether the Reader is part of a consumer group.
func (r *Reader) useConsumerGroup() bool { return r.config.GroupID != "" }
// useSyncCommits indicates whether the Reader is configured to perform sync or
// async commits.
func (r *Reader) useSyncCommits() bool { return r.config.CommitInterval == 0 }
// membership returns the group generationID and memberID of the reader.
//
// Only used when config.GroupID != ""
func (r *Reader) membership() (generationID int32, memberID string) {
r.mutex.Lock()
generationID = r.generationID
memberID = r.memberID
r.mutex.Unlock()
return
}
// lookupCoordinator scans the brokers and looks up the address of the
// coordinator for the group.
//
// Only used when config.GroupID != ""
func (r *Reader) lookupCoordinator() (string, error) {
conn, err := r.connect()
if err != nil {
return "", fmt.Errorf("unable to coordinator to any connect for group, %v: %v\n", r.config.GroupID, err)
}
defer conn.Close()
out, err := conn.findCoordinator(findCoordinatorRequestV0{
CoordinatorKey: r.config.GroupID,
})
if err != nil {
return "", fmt.Errorf("unable to find coordinator for group, %v: %v", r.config.GroupID, err)
}
address := fmt.Sprintf("%v:%v", out.Coordinator.Host, out.Coordinator.Port)
return address, nil
}
// refreshCoordinator updates the value of r.address
func (r *Reader) refreshCoordinator() (err error) {
const (
backoffDelayMin = 100 * time.Millisecond
backoffDelayMax = 1 * time.Second
)
for attempt := 0; true; attempt++ {
if attempt != 0 {
if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) {
return r.stctx.Err()
}
}
address, err := r.lookupCoordinator()
if err != nil {
continue
}
r.mutex.Lock()
oldAddress := r.address
r.address = address
r.mutex.Unlock()
if address != oldAddress {
r.withLogger(func(l *log.Logger) {
l.Printf("coordinator for group, %v, set to %v\n", r.config.GroupID, address)
})
}
break
}
return nil
}
// makejoinGroupRequestV1 handles the logic of constructing a joinGroup
// request
func (r *Reader) makejoinGroupRequestV1() (joinGroupRequestV1, error) {
_, memberID := r.membership()
request := joinGroupRequestV1{
GroupID: r.config.GroupID,
MemberID: memberID,
SessionTimeout: int32(r.config.SessionTimeout / time.Millisecond),
RebalanceTimeout: int32(r.config.RebalanceTimeout / time.Millisecond),
ProtocolType: defaultProtocolType,
}
for _, balancer := range r.config.GroupBalancers {
userData, err := balancer.UserData()
if err != nil {
return joinGroupRequestV1{}, fmt.Errorf("unable to construct protocol metadata for member, %v: %v\n", balancer.ProtocolName(), err)
}
request.GroupProtocols = append(request.GroupProtocols, joinGroupRequestGroupProtocolV1{
ProtocolName: balancer.ProtocolName(),
ProtocolMetadata: groupMetadata{
Version: 1,
Topics: []string{r.config.Topic},
UserData: userData,
}.bytes(),
})
}
return request, nil
}
// makeMemberProtocolMetadata maps encoded member metadata ([]byte) into []GroupMember
func (r *Reader) makeMemberProtocolMetadata(in []joinGroupResponseMemberV1) ([]GroupMember, error) {
members := make([]GroupMember, 0, len(in))
for _, item := range in {
metadata := groupMetadata{}
reader := bufio.NewReader(bytes.NewReader(item.MemberMetadata))
if remain, err := (&metadata).readFrom(reader, len(item.MemberMetadata)); err != nil || remain != 0 {
return nil, fmt.Errorf("unable to read metadata for member, %v: %v\n", item.MemberID, err)
}
members = append(members, GroupMember{
ID: item.MemberID,
Topics: metadata.Topics,
UserData: metadata.UserData,
})
}
return members, nil
}
// partitionReader is an internal interface used to simplify unit testing
type partitionReader interface {
// ReadPartitions mirrors Conn.ReadPartitions
ReadPartitions(topics ...string) (partitions []Partition, err error)
}
// assignTopicPartitions uses the selected GroupBalancer to assign members to
// their various partitions
func (r *Reader) assignTopicPartitions(conn partitionReader, group joinGroupResponseV1) (GroupMemberAssignments, error) {
r.withLogger(func(l *log.Logger) {
l.Println("selected as leader for group,", r.config.GroupID)
})
balancer, ok := findGroupBalancer(group.GroupProtocol, r.config.GroupBalancers)
if !ok {
return nil, fmt.Errorf("unable to find selected balancer, %v, for group, %v", group.GroupProtocol, r.config.GroupID)
}
members, err := r.makeMemberProtocolMetadata(group.Members)
if err != nil {
return nil, fmt.Errorf("unable to construct MemberProtocolMetadata: %v", err)
}
topics := extractTopics(members)
partitions, err := conn.ReadPartitions(topics...)
// it's not a failure if the topic doesn't exist yet. it results in no
// assignments for the topic. this matches the behavior of the official
// clients: java, python, and librdkafka.
// a topic watcher can trigger a rebalance when the topic comes into being.
if err != nil && err != UnknownTopicOrPartition {
return nil, fmt.Errorf("unable to read partitions: %v", err)
}
r.withLogger(func(l *log.Logger) {
l.Printf("using '%v' balancer to assign group, %v\n", group.GroupProtocol, r.config.GroupID)
for _, member := range members {
l.Printf("found member: %v/%#v", member.ID, member.UserData)
}
for _, partition := range partitions {
l.Printf("found topic/partition: %v/%v", partition.Topic, partition.ID)
}
})
return balancer.AssignGroups(members, partitions), nil
}
func (r *Reader) leaveGroup(conn *Conn) error {
_, memberID := r.membership()
_, err := conn.leaveGroup(leaveGroupRequestV0{
GroupID: r.config.GroupID,
MemberID: memberID,
})
if err != nil {
return fmt.Errorf("leave group failed for group, %v, and member, %v: %v", r.config.GroupID, memberID, err)
}
return nil
}
// joinGroup attempts to join the reader to the consumer group.
// Returns GroupMemberAssignments is this Reader was selected as
// the leader. Otherwise, GroupMemberAssignments will be nil.
//
// Possible kafka error codes returned:
// * GroupLoadInProgress:
// * GroupCoordinatorNotAvailable:
// * NotCoordinatorForGroup:
// * InconsistentGroupProtocol:
// * InvalidSessionTimeout:
// * GroupAuthorizationFailed:
func (r *Reader) joinGroup(conn *Conn) (GroupMemberAssignments, error) {
request, err := r.makejoinGroupRequestV1()
if err != nil {
return nil, err
}
response, err := conn.joinGroup(request)
if err != nil {
switch err {
case UnknownMemberId:
r.mutex.Lock()
r.memberID = ""
r.mutex.Unlock()
return nil, fmt.Errorf("joinGroup failed: %v", err)
default:
return nil, fmt.Errorf("joinGroup failed: %v", err)
}
}
// Extract our membership and generationID from the response
r.mutex.Lock()
oldGenerationID := r.generationID
oldMemberID := r.memberID
r.generationID = response.GenerationID
r.memberID = response.MemberID
r.mutex.Unlock()
if oldGenerationID != response.GenerationID || oldMemberID != response.MemberID {
r.withLogger(func(l *log.Logger) {
l.Printf("response membership changed. generationID: %v => %v, memberID: '%v' => '%v'\n",
oldGenerationID,
response.GenerationID,
oldMemberID,
response.MemberID,
)
})
}
var assignments GroupMemberAssignments
if iAmLeader := response.MemberID == response.LeaderID; iAmLeader {
v, err := r.assignTopicPartitions(conn, response)
if err != nil {
_ = r.leaveGroup(conn)
return nil, err
}
assignments = v
r.withLogger(func(l *log.Logger) {
for memberID, assignment := range assignments {
for topic, partitions := range assignment {
l.Printf("assigned member/topic/partitions %v/%v/%v\n", memberID, topic, partitions)
}
}
})
}
r.withLogger(func(l *log.Logger) {
l.Printf("joinGroup succeeded for response, %v. generationID=%v, memberID=%v\n", r.config.GroupID, response.GenerationID, response.MemberID)
})
return assignments, nil
}
func (r *Reader) makeSyncGroupRequestV0(memberAssignments GroupMemberAssignments) syncGroupRequestV0 {
generationID, memberID := r.membership()
request := syncGroupRequestV0{
GroupID: r.config.GroupID,
GenerationID: generationID,
MemberID: memberID,
}
if memberAssignments != nil {
request.GroupAssignments = make([]syncGroupRequestGroupAssignmentV0, 0, 1)
for memberID, topics := range memberAssignments {
topics32 := make(map[string][]int32)
for topic, partitions := range topics {
partitions32 := make([]int32, len(partitions))
for i := range partitions {
partitions32[i] = int32(partitions[i])
}
topics32[topic] = partitions32
}
request.GroupAssignments = append(request.GroupAssignments, syncGroupRequestGroupAssignmentV0{
MemberID: memberID,
MemberAssignments: groupAssignment{
Version: 1,
Topics: topics32,
}.bytes(),
})
}
r.withErrorLogger(func(logger *log.Logger) {
logger.Printf("Syncing %d assignments for generation %d as member %s", len(request.GroupAssignments), generationID, memberID)
})
}
return request
}
// syncGroup completes the consumer group handshake by accepting the
// memberAssignments (if this Reader is the leader) and returning this
// Readers subscriptions topic => partitions
//
// Possible kafka error codes returned:
// * GroupCoordinatorNotAvailable:
// * NotCoordinatorForGroup:
// * IllegalGeneration:
// * RebalanceInProgress:
// * GroupAuthorizationFailed:
func (r *Reader) syncGroup(conn *Conn, memberAssignments GroupMemberAssignments) (map[string][]int32, error) {
request := r.makeSyncGroupRequestV0(memberAssignments)
response, err := conn.syncGroups(request)
if err != nil {
switch err {
case RebalanceInProgress:
// don't leave the group
return nil, fmt.Errorf("syncGroup failed: %v", err)
case UnknownMemberId:
r.mutex.Lock()
r.memberID = ""
r.mutex.Unlock()
_ = r.leaveGroup(conn)
return nil, fmt.Errorf("syncGroup failed: %v", err)
default:
_ = r.leaveGroup(conn)
return nil, fmt.Errorf("syncGroup failed: %v", err)
}
}
assignments := groupAssignment{}
reader := bufio.NewReader(bytes.NewReader(response.MemberAssignments))
if _, err := (&assignments).readFrom(reader, len(response.MemberAssignments)); err != nil {
_ = r.leaveGroup(conn)
return nil, fmt.Errorf("unable to read SyncGroup response for group, %v: %v\n", r.config.GroupID, err)
}
if len(assignments.Topics) == 0 {
generation, memberID := r.membership()
r.withLogger(func(l *log.Logger) {
l.Printf("received empty assignments for group, %v as member %s for generation %d", r.config.GroupID, memberID, generation)
})
}
r.withLogger(func(l *log.Logger) {
l.Printf("sync group finished for group, %v\n", r.config.GroupID)
})
return assignments.Topics, nil
}
func (r *Reader) rebalance(conn *Conn) (map[string][]int32, error) {
r.stats.rebalances.observe(1)
r.withLogger(func(l *log.Logger) {
l.Printf("rebalancing consumer group, %v", r.config.GroupID)
})
members, err := r.joinGroup(conn)
if err != nil {
return nil, err
}
assignments, err := r.syncGroup(conn, members)
if err != nil {
return nil, err
}
return assignments, nil
}
func (r *Reader) unsubscribe() error {
r.cancel()
r.join.Wait()
return nil
}
func (r *Reader) fetchOffsets(conn *Conn, subs map[string][]int32) (map[int]int64, error) {
partitions := subs[r.config.Topic]
offsets, err := conn.offsetFetch(offsetFetchRequestV1{
GroupID: r.config.GroupID,
Topics: []offsetFetchRequestV1Topic{
{
Topic: r.config.Topic,
Partitions: partitions,
},
},
})
if err != nil {
return nil, err
}
offsetsByPartition := map[int]int64{}
for _, pr := range offsets.Responses[0].PartitionResponses {
for _, partition := range partitions {
if partition == pr.Partition {
offset := pr.Offset
if offset < 0 {
// No offset stored
offset = FirstOffset
}
offsetsByPartition[int(partition)] = offset
}
}
}
return offsetsByPartition, nil
}
func (r *Reader) subscribe(conn *Conn, subs map[string][]int32) error {
if len(subs[r.config.Topic]) == 0 {
return nil
}
offsetsByPartition, err := r.fetchOffsets(conn, subs)
if err != nil {
return err
}
r.mutex.Lock()
r.start(offsetsByPartition)
r.mutex.Unlock()
r.withLogger(func(l *log.Logger) {
l.Printf("subscribed to partitions: %+v", offsetsByPartition)
})
return nil
}
// connect returns a connection to ANY broker
func (r *Reader) connect() (conn *Conn, err error) {
for _, broker := range r.config.Brokers {
if conn, err = r.config.Dialer.Dial("tcp", broker); err == nil {
return
}
}
return // err will be non-nil
}
// coordinator returns a connection to the coordinator for this group
func (r *Reader) coordinator() (*Conn, error) {
r.mutex.Lock()
address := r.address
r.mutex.Unlock()
conn, err := r.config.Dialer.DialContext(r.stctx, "tcp", address)
if err != nil {
return nil, fmt.Errorf("unable to connect to coordinator, %v", address)
}
return conn, nil
}
func (r *Reader) waitThrottleTime(throttleTimeMS int32) {
if throttleTimeMS == 0 {
return
}
t := time.NewTimer(time.Duration(throttleTimeMS) * time.Millisecond)
defer t.Stop()
select {
case <-r.stctx.Done():
return
case <-t.C:
}
}
// heartbeat sends heartbeat to coordinator at the interval defined by
// ReaderConfig.HeartbeatInterval
func (r *Reader) heartbeat(conn *Conn) error {
generationID, memberID := r.membership()
if generationID == 0 && memberID == "" {
return nil
}
_, err := conn.heartbeat(heartbeatRequestV0{
GroupID: r.config.GroupID,
GenerationID: generationID,
MemberID: memberID,
})
if err != nil {
return fmt.Errorf("heartbeat failed: %v", err)
}
return nil
}
func (r *Reader) heartbeatLoop(conn *Conn) func(stop <-chan struct{}) {
return func(stop <-chan struct{}) {
r.withLogger(func(l *log.Logger) {
l.Printf("started heartbeat for group, %v [%v]", r.config.GroupID, r.config.HeartbeatInterval)
})
defer r.withLogger(func(l *log.Logger) {
l.Println("stopped heartbeat for group,", r.config.GroupID)
})
ticker := time.NewTicker(r.config.HeartbeatInterval)
defer ticker.Stop()
for {
select {
case <-ticker.C:
if err := r.heartbeat(conn); err != nil {
return
}
case <-stop:
return
}
}
}
}
type offsetCommitter interface {
offsetCommit(request offsetCommitRequestV2) (offsetCommitResponseV2, error)
}
func (r *Reader) commitOffsets(conn offsetCommitter, offsetStash offsetStash) error {
if len(offsetStash) == 0 {
return nil
}
generationID, memberID := r.membership()
request := offsetCommitRequestV2{
GroupID: r.config.GroupID,
GenerationID: generationID,
MemberID: memberID,
RetentionTime: int64(r.config.RetentionTime / time.Millisecond),
}
for topic, partitions := range offsetStash {
t := offsetCommitRequestV2Topic{Topic: topic}
for partition, offset := range partitions {
t.Partitions = append(t.Partitions, offsetCommitRequestV2Partition{
Partition: int32(partition),
Offset: offset,
})
}
request.Topics = append(request.Topics, t)
}
if _, err := conn.offsetCommit(request); err != nil {
return fmt.Errorf("unable to commit offsets for group, %v: %v", r.config.GroupID, err)
}
r.withLogger(func(l *log.Logger) {
l.Printf("committed offsets: %v", offsetStash)
})
return nil
}
// commitOffsetsWithRetry attempts to commit the specified offsets and retries
// up to the specified number of times
func (r *Reader) commitOffsetsWithRetry(conn offsetCommitter, offsetStash offsetStash, retries int) (err error) {
const (
backoffDelayMin = 100 * time.Millisecond
backoffDelayMax = 5 * time.Second
)
for attempt := 0; attempt < retries; attempt++ {
if attempt != 0 {
if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) {
return
}
}
if err = r.commitOffsets(conn, offsetStash); err == nil {
return
}
}
return // err will not be nil
}
// offsetStash holds offsets by topic => partition => offset
type offsetStash map[string]map[int]int64
// merge updates the offsetStash with the offsets from the provided messages
func (o offsetStash) merge(commits []commit) {
for _, c := range commits {
offsetsByPartition, ok := o[c.topic]
if !ok {
offsetsByPartition = map[int]int64{}
o[c.topic] = offsetsByPartition
}
if offset, ok := offsetsByPartition[c.partition]; !ok || c.offset > offset {
offsetsByPartition[c.partition] = c.offset
}
}
}
// reset clears the contents of the offsetStash
func (o offsetStash) reset() {
for key := range o {
delete(o, key)
}
}
// commitLoopImmediate handles each commit synchronously
func (r *Reader) commitLoopImmediate(conn offsetCommitter, stop <-chan struct{}) {
offsetsByTopicAndPartition := offsetStash{}
for {
select {
case <-stop:
return
case req := <-r.commits:
offsetsByTopicAndPartition.merge(req.commits)
req.errch <- r.commitOffsetsWithRetry(conn, offsetsByTopicAndPartition, defaultCommitRetries)
offsetsByTopicAndPartition.reset()
}
}
}
// commitLoopInterval handles each commit asynchronously with a period defined
// by ReaderConfig.CommitInterval
func (r *Reader) commitLoopInterval(conn offsetCommitter, stop <-chan struct{}) {
ticker := time.NewTicker(r.config.CommitInterval)
defer ticker.Stop()
commit := func() {
if err := r.commitOffsetsWithRetry(conn, r.offsetStash, defaultCommitRetries); err != nil {
r.withErrorLogger(func(l *log.Logger) { l.Print(err) })
} else {
r.offsetStash.reset()
}
}
for {
select {
case <-stop:
// drain the commit channel in order to prepare the final commit.
for hasCommits := true; hasCommits; {
select {
case req := <-r.commits:
r.offsetStash.merge(req.commits)
default:
hasCommits = false
}
}
commit()
return
case <-ticker.C:
commit()
case req := <-r.commits:
r.offsetStash.merge(req.commits)
}
}
}
// commitLoop processes commits off the commit chan
func (r *Reader) commitLoop(conn *Conn) func(stop <-chan struct{}) {
return func(stop <-chan struct{}) {
r.withLogger(func(l *log.Logger) {
l.Println("started commit for group,", r.config.GroupID)
})
defer r.withLogger(func(l *log.Logger) {
l.Println("stopped commit for group,", r.config.GroupID)
})
if r.config.CommitInterval == 0 {
r.commitLoopImmediate(conn, stop)
} else {
r.commitLoopInterval(conn, stop)
}
}
}
// partitionWatcher queries kafka and watches for partition changes, triggering a rebalance if changes are found.
// Similar to heartbeat it's okay to return on error here as if you are unable to ask a broker for basic metadata
// you're in a bad spot and should rebalance. Commonly you will see an error here if there is a problem with
// the connection to the coordinator and a rebalance will establish a new connection to the coordinator.
func (r *Reader) partitionWatcher(conn partitionReader) func(stop <-chan struct{}) {
return func(stop <-chan struct{}) {
ticker := time.NewTicker(r.config.PartitionWatchInterval)
defer ticker.Stop()
ops, err := conn.ReadPartitions(r.config.Topic)
if err != nil {
r.withErrorLogger(func(l *log.Logger) {
l.Printf("Problem getting partitions during startup, %v\n, Returning and setting up handshake", err)
})
return
}
oParts := len(ops)
for {
select {
case <-stop:
return
case <-ticker.C:
ops, err := conn.ReadPartitions(r.config.Topic)
if err != nil {
r.withErrorLogger(func(l *log.Logger) {
l.Printf("Problem getting partitions while checking for changes, %v\n", err)
})
return
}
if len(ops) != oParts {
r.withErrorLogger(func(l *log.Logger) {
l.Printf("Partition changes found, reblancing group: %v.", r.config.GroupID)
})
return
}
}
}
}
}
// handshake performs the necessary incantations to join this Reader to the desired
// consumer group. handshake will be called whenever the group is disrupted
// (member join, member leave, coordinator changed, etc)
func (r *Reader) handshake() error {
// always clear prior to subscribe
r.unsubscribe()
// make sure we have the most up-to-date coordinator.
if err := r.refreshCoordinator(); err != nil {
return err
}
// establish a connection to the coordinator. this connection will be
// shared by all of the consumer group go routines.
conn, err := r.coordinator()
if err != nil {
return err
}
defer func() {
select {
case <-r.stctx.Done():
// this reader is closing...leave the consumer group.
_ = r.leaveGroup(conn)
default:
// another consumer has left the group
}
_ = conn.Close()
}()
// rebalance and fetch assignments
assignments, err := r.rebalance(conn)
if err != nil {
return fmt.Errorf("rebalance failed for consumer group, %v: %v", r.config.GroupID, err)
}
rg := &runGroup{}
rg = rg.WithContext(r.stctx)
rg.Go(r.heartbeatLoop(conn))
rg.Go(r.commitLoop(conn))
if r.config.WatchPartitionChanges {
rg.Go(r.partitionWatcher(conn))
}
// subscribe to assignments
if err := r.subscribe(conn, assignments); err != nil {
rg.Stop()
return fmt.Errorf("subscribe failed for consumer group, %v: %v\n", r.config.GroupID, err)
}
rg.Wait()
return nil
}
// run provides the main consumer group management loop. Each iteration performs the
// handshake to join the Reader to the consumer group.
func (r *Reader) run() {
defer close(r.done)
if !r.useConsumerGroup() {
return
}
r.withLogger(func(l *log.Logger) {
l.Printf("entering loop for consumer group, %v\n", r.config.GroupID)
})
for {
if err := r.handshake(); err != nil {
r.stats.errors.observe(1)
r.withErrorLogger(func(l *log.Logger) {
l.Println(err)
})
}
select {
case <-r.stctx.Done():
return
default:
}
}
}
// ReaderConfig is a configuration object used to create new instances of
// Reader.
type ReaderConfig struct {
// The list of broker addresses used to connect to the kafka cluster.
Brokers []string
// GroupID holds the optional consumer group id. If GroupID is specified, then
// Partition should NOT be specified e.g. 0
GroupID string
// The topic to read messages from.
Topic string
// Partition to read messages from. Either Partition or GroupID may
// be assigned, but not both
Partition int
// An dialer used to open connections to the kafka server. This field is
// optional, if nil, the default dialer is used instead.
Dialer *Dialer
// The capacity of the internal message queue, defaults to 100 if none is
// set.
QueueCapacity int
// Min and max number of bytes to fetch from kafka in each request.
MinBytes int
MaxBytes int
// Maximum amount of time to wait for new data to come when fetching batches
// of messages from kafka.
MaxWait time.Duration
// ReadLagInterval sets the frequency at which the reader lag is updated.
// Setting this field to a negative value disables lag reporting.
ReadLagInterval time.Duration
// GroupBalancers is the priority-ordered list of client-side consumer group
// balancing strategies that will be offered to the coordinator. The first
// strategy that all group members support will be chosen by the leader.
//
// Default: [Range, RoundRobin]
//
// Only used when GroupID is set
GroupBalancers []GroupBalancer
// HeartbeatInterval sets the optional frequency at which the reader sends the consumer
// group heartbeat update.
//
// Default: 3s
//
// Only used when GroupID is set
HeartbeatInterval time.Duration
// CommitInterval indicates the interval at which offsets are committed to
// the broker. If 0, commits will be handled synchronously.
//
// Default: 0
//
// Only used when GroupID is set
CommitInterval time.Duration
// PartitionWatchInterval indicates how often a reader checks for partition changes.
// If a reader sees a partition change (such as a partition add) it will rebalance the group
// picking up new partitions.
//
// Default: 5s
//
// Only used when GroupID is set and WatchPartitionChanges is set.
PartitionWatchInterval time.Duration
// WatchForPartitionChanges is used to inform kafka-go that a consumer group should be
// polling the brokers and rebalancing if any partition changes happen to the topic.
WatchPartitionChanges bool
// SessionTimeout optionally sets the length of time that may pass without a heartbeat
// before the coordinator considers the consumer dead and initiates a rebalance.
//
// Default: 30s
//
// Only used when GroupID is set
SessionTimeout time.Duration
// RebalanceTimeout optionally sets the length of time the coordinator will wait
// for members to join as part of a rebalance. For kafka servers under higher
// load, it may be useful to set this value higher.
//
// Default: 30s
//
// Only used when GroupID is set
RebalanceTimeout time.Duration
// RetentionTime optionally sets the length of time the consumer group will be saved
// by the broker
//