-
Notifications
You must be signed in to change notification settings - Fork 789
/
reader.go
1603 lines (1361 loc) · 45.2 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 (
"context"
"errors"
"fmt"
"io"
"math"
"sort"
"strconv"
"sync"
"sync/atomic"
"time"
)
const (
LastOffset int64 = -1 // The most recent offset available for a partition.
FirstOffset int64 = -2 // The least recent offset available for a partition.
)
const (
// defaultCommitRetries holds the number commit attempts to make
// before giving up.
defaultCommitRetries = 3
)
const (
// defaultFetchMinBytes of 1 byte means that fetch requests are answered as
// soon as a single byte of data is available or the fetch request times out
// waiting for data to arrive.
defaultFetchMinBytes = 1
)
var (
errOnlyAvailableWithGroup = errors.New("unavailable when GroupID is not set")
errNotAvailableWithGroup = errors.New("unavailable when GroupID is set")
)
const (
// 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.
//
// Note that it is important to call `Close()` on a `Reader` when a process exits.
// The kafka server needs a graceful disconnect to stop it from continuing to
// attempt to send messages to the connected clients. The given example will not
// call `Close()` if the process is terminated with SIGINT (ctrl-c at the shell) or
// SIGTERM (as docker stop or a kubernetes restart does). This can result in a
// delay when a new reader on the same topic connects (e.g. new process started
// or new container running). Use a `signal.Notify` handler to close the reader on
// process shutdown.
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
// Without a group subscription (when Reader.config.GroupID == ""),
// when errors occur, the Reader gets a synthetic readerMessage with
// a non-nil err set. With group subscriptions however, when an error
// occurs in Reader.run, there's no reader running (sic, cf. reader vs.
// Reader) and there's no way to let the high-level methods like
// FetchMessage know that an error indeed occurred. If an error in run
// occurs, it will be non-block-sent to this unbuffered channel, where
// the high-level methods can select{} on it and notify the caller.
runError chan error
// 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 != "" }
func (r *Reader) getTopics() []string {
if len(r.config.GroupTopics) > 0 {
return r.config.GroupTopics[:]
}
return []string{r.config.Topic}
}
// useSyncCommits indicates whether the Reader is configured to perform sync or
// async commits.
func (r *Reader) useSyncCommits() bool { return r.config.CommitInterval == 0 }
func (r *Reader) unsubscribe() {
r.cancel()
r.join.Wait()
// it would be interesting to drain the r.msgs channel at this point since
// it will contain buffered messages for partitions that may not be
// re-assigned to this reader in the next consumer group generation.
// however, draining the channel could race with the client calling
// ReadMessage, which could result in messages delivered and/or committed
// with gaps in the offset. for now, we will err on the side of caution and
// potentially have those messages be reprocessed in the next generation by
// another consumer to avoid such a race.
}
func (r *Reader) subscribe(allAssignments map[string][]PartitionAssignment) {
offsets := make(map[topicPartition]int64)
for topic, assignments := range allAssignments {
for _, assignment := range assignments {
key := topicPartition{
topic: topic,
partition: int32(assignment.ID),
}
offsets[key] = assignment.Offset
}
}
r.mutex.Lock()
r.start(offsets)
r.mutex.Unlock()
r.withLogger(func(l Logger) {
l.Printf("subscribed to topics and partitions: %+v", offsets)
})
}
// commitOffsetsWithRetry attempts to commit the specified offsets and retries
// up to the specified number of times.
func (r *Reader) commitOffsetsWithRetry(gen *Generation, 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 = gen.CommitOffsets(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(ctx context.Context, gen *Generation) {
offsets := offsetStash{}
for {
select {
case <-ctx.Done():
// drain the commit channel and prepare a single, final commit.
// the commit will combine any outstanding requests and the result
// will be sent back to all the callers of CommitMessages so that
// they can return.
var errchs []chan<- error
for hasCommits := true; hasCommits; {
select {
case req := <-r.commits:
offsets.merge(req.commits)
errchs = append(errchs, req.errch)
default:
hasCommits = false
}
}
err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries)
for _, errch := range errchs {
// NOTE : this will be a buffered channel and will not block.
errch <- err
}
return
case req := <-r.commits:
offsets.merge(req.commits)
req.errch <- r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries)
offsets.reset()
}
}
}
// commitLoopInterval handles each commit asynchronously with a period defined
// by ReaderConfig.CommitInterval.
func (r *Reader) commitLoopInterval(ctx context.Context, gen *Generation) {
ticker := time.NewTicker(r.config.CommitInterval)
defer ticker.Stop()
// the offset stash should not survive rebalances b/c the consumer may
// receive new assignments.
offsets := offsetStash{}
commit := func() {
if err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries); err != nil {
r.withErrorLogger(func(l Logger) { l.Printf(err.Error()) })
} else {
offsets.reset()
}
}
for {
select {
case <-ctx.Done():
// drain the commit channel in order to prepare the final commit.
for hasCommits := true; hasCommits; {
select {
case req := <-r.commits:
offsets.merge(req.commits)
default:
hasCommits = false
}
}
commit()
return
case <-ticker.C:
commit()
case req := <-r.commits:
offsets.merge(req.commits)
}
}
}
// commitLoop processes commits off the commit chan.
func (r *Reader) commitLoop(ctx context.Context, gen *Generation) {
r.withLogger(func(l Logger) {
l.Printf("started commit for group %s\n", r.config.GroupID)
})
defer r.withLogger(func(l Logger) {
l.Printf("stopped commit for group %s\n", r.config.GroupID)
})
if r.config.CommitInterval == 0 {
r.commitLoopImmediate(ctx, gen)
} else {
r.commitLoopInterval(ctx, gen)
}
}
// run provides the main consumer group management loop. Each iteration performs the
// handshake to join the Reader to the consumer group.
//
// This function is responsible for closing the consumer group upon exit.
func (r *Reader) run(cg *ConsumerGroup) {
defer close(r.done)
defer cg.Close()
r.withLogger(func(l Logger) {
l.Printf("entering loop for consumer group, %v\n", r.config.GroupID)
})
for {
// Limit the number of attempts at waiting for the next
// consumer generation.
var err error
var gen *Generation
for attempt := 1; attempt <= r.config.MaxAttempts; attempt++ {
gen, err = cg.Next(r.stctx)
if err == nil {
break
}
if errors.Is(err, r.stctx.Err()) {
return
}
r.stats.errors.observe(1)
r.withErrorLogger(func(l Logger) {
l.Printf(err.Error())
})
// Continue with next attempt...
}
if err != nil {
// All attempts have failed.
select {
case r.runError <- err:
// If somebody's receiving on the runError, let
// them know the error occurred.
default:
// Otherwise, don't block to allow healing.
}
continue
}
r.stats.rebalances.observe(1)
r.subscribe(gen.Assignments)
gen.Start(func(ctx context.Context) {
r.commitLoop(ctx, gen)
})
gen.Start(func(ctx context.Context) {
// wait for the generation to end and then unsubscribe.
select {
case <-ctx.Done():
// continue to next generation
case <-r.stctx.Done():
// this will be the last loop because the reader is closed.
}
r.unsubscribe()
})
}
}
// 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
// GroupTopics allows specifying multiple topics, but can only be used in
// combination with GroupID, as it is a consumer-group feature. As such, if
// GroupID is set, then either Topic or GroupTopics must be defined.
GroupTopics []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
// MinBytes indicates to the broker the minimum batch size that the consumer
// will accept. Setting a high minimum when consuming from a low-volume topic
// may result in delayed delivery when the broker does not have enough data to
// satisfy the defined minimum.
//
// Default: 1
MinBytes int
// MaxBytes indicates to the broker the maximum batch size that the consumer
// will accept. The broker will truncate a message to satisfy this maximum, so
// choose a value that is high enough for your largest message size.
//
// Default: 1MB
MaxBytes int
// Maximum amount of time to wait for new data to come when fetching batches
// of messages from kafka.
//
// Default: 10s
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
// JoinGroupBackoff optionally sets the length of time to wait between re-joining
// the consumer group after an error.
//
// Default: 5s
JoinGroupBackoff time.Duration
// RetentionTime optionally sets the length of time the consumer group will be saved
// by the broker
//
// Default: 24h
//
// Only used when GroupID is set
RetentionTime time.Duration
// StartOffset determines from whence the consumer group should begin
// consuming when it finds a partition without a committed offset. If
// non-zero, it must be set to one of FirstOffset or LastOffset.
//
// Default: FirstOffset
//
// Only used when GroupID is set
StartOffset int64
// BackoffDelayMin optionally sets the smallest amount of time the reader will wait before
// polling for new messages
//
// Default: 100ms
ReadBackoffMin time.Duration
// BackoffDelayMax optionally sets the maximum amount of time the reader will wait before
// polling for new messages
//
// Default: 1s
ReadBackoffMax time.Duration
// If not nil, specifies a logger used to report internal changes within the
// reader.
Logger Logger
// ErrorLogger is the logger used to report errors. If nil, the reader falls
// back to using Logger instead.
ErrorLogger Logger
// IsolationLevel controls the visibility of transactional records.
// ReadUncommitted makes all records visible. With ReadCommitted only
// non-transactional and committed records are visible.
IsolationLevel IsolationLevel
// Limit of how many attempts will be made before delivering the error.
//
// The default is to try 3 times.
MaxAttempts int
// OffsetOutOfRangeError indicates that the reader should return an error in
// the event of an OffsetOutOfRange error, rather than retrying indefinitely.
// This flag is being added to retain backwards-compatibility, so it will be
// removed in a future version of kafka-go.
OffsetOutOfRangeError bool
}
// Validate method validates ReaderConfig properties.
func (config *ReaderConfig) Validate() error {
if len(config.Brokers) == 0 {
return errors.New("cannot create a new kafka reader with an empty list of broker addresses")
}
if config.Partition < 0 || config.Partition >= math.MaxInt32 {
return fmt.Errorf("partition number out of bounds: %d", config.Partition)
}
if config.MinBytes < 0 {
return fmt.Errorf("invalid negative minimum batch size (min = %d)", config.MinBytes)
}
if config.MaxBytes < 0 {
return fmt.Errorf("invalid negative maximum batch size (max = %d)", config.MaxBytes)
}
if config.GroupID != "" {
if config.Partition != 0 {
return errors.New("either Partition or GroupID may be specified, but not both")
}
if len(config.Topic) == 0 && len(config.GroupTopics) == 0 {
return errors.New("either Topic or GroupTopics must be specified with GroupID")
}
} else if len(config.Topic) == 0 {
return errors.New("cannot create a new kafka reader with an empty topic")
}
if config.MinBytes > config.MaxBytes {
return fmt.Errorf("minimum batch size greater than the maximum (min = %d, max = %d)", config.MinBytes, config.MaxBytes)
}
if config.ReadBackoffMax < 0 {
return fmt.Errorf("ReadBackoffMax out of bounds: %d", config.ReadBackoffMax)
}
if config.ReadBackoffMin < 0 {
return fmt.Errorf("ReadBackoffMin out of bounds: %d", config.ReadBackoffMin)
}
return nil
}
// ReaderStats is a data structure returned by a call to Reader.Stats that exposes
// details about the behavior of the reader.
type ReaderStats struct {
Dials int64 `metric:"kafka.reader.dial.count" type:"counter"`
Fetches int64 `metric:"kafka.reader.fetch.count" type:"counter"`
Messages int64 `metric:"kafka.reader.message.count" type:"counter"`
Bytes int64 `metric:"kafka.reader.message.bytes" type:"counter"`
Rebalances int64 `metric:"kafka.reader.rebalance.count" type:"counter"`
Timeouts int64 `metric:"kafka.reader.timeout.count" type:"counter"`
Errors int64 `metric:"kafka.reader.error.count" type:"counter"`
DialTime DurationStats `metric:"kafka.reader.dial.seconds"`
ReadTime DurationStats `metric:"kafka.reader.read.seconds"`
WaitTime DurationStats `metric:"kafka.reader.wait.seconds"`
FetchSize SummaryStats `metric:"kafka.reader.fetch.size"`
FetchBytes SummaryStats `metric:"kafka.reader.fetch.bytes"`
Offset int64 `metric:"kafka.reader.offset" type:"gauge"`
Lag int64 `metric:"kafka.reader.lag" type:"gauge"`
MinBytes int64 `metric:"kafka.reader.fetch_bytes.min" type:"gauge"`
MaxBytes int64 `metric:"kafka.reader.fetch_bytes.max" type:"gauge"`
MaxWait time.Duration `metric:"kafka.reader.fetch_wait.max" type:"gauge"`
QueueLength int64 `metric:"kafka.reader.queue.length" type:"gauge"`
QueueCapacity int64 `metric:"kafka.reader.queue.capacity" type:"gauge"`
ClientID string `tag:"client_id"`
Topic string `tag:"topic"`
Partition string `tag:"partition"`
// The original `Fetches` field had a typo where the metric name was called
// "kafak..." instead of "kafka...", in order to offer time to fix monitors
// that may be relying on this mistake we are temporarily introducing this
// field.
DeprecatedFetchesWithTypo int64 `metric:"kafak.reader.fetch.count" type:"counter"`
}
// readerStats is a struct that contains statistics on a reader.
type readerStats struct {
dials counter
fetches counter
messages counter
bytes counter
rebalances counter
timeouts counter
errors counter
dialTime summary
readTime summary
waitTime summary
fetchSize summary
fetchBytes summary
offset gauge
lag gauge
partition string
}
// NewReader creates and returns a new Reader configured with config.
// The offset is initialized to FirstOffset.
func NewReader(config ReaderConfig) *Reader {
if err := config.Validate(); err != nil {
panic(err)
}
if config.GroupID != "" {
if len(config.GroupBalancers) == 0 {
config.GroupBalancers = []GroupBalancer{
RangeGroupBalancer{},
RoundRobinGroupBalancer{},
}
}
}
if config.Dialer == nil {
config.Dialer = DefaultDialer
}
if config.MaxBytes == 0 {
config.MaxBytes = 1e6 // 1 MB
}
if config.MinBytes == 0 {
config.MinBytes = defaultFetchMinBytes
}
if config.MaxWait == 0 {
config.MaxWait = 10 * time.Second
}
if config.ReadLagInterval == 0 {
config.ReadLagInterval = 1 * time.Minute
}
if config.ReadBackoffMin == 0 {
config.ReadBackoffMin = defaultReadBackoffMin
}
if config.ReadBackoffMax == 0 {
config.ReadBackoffMax = defaultReadBackoffMax
}
if config.ReadBackoffMax < config.ReadBackoffMin {
panic(fmt.Errorf("ReadBackoffMax %d smaller than ReadBackoffMin %d", config.ReadBackoffMax, config.ReadBackoffMin))
}
if config.QueueCapacity == 0 {
config.QueueCapacity = 100
}
if config.MaxAttempts == 0 {
config.MaxAttempts = 3
}
// when configured as a consumer group; stats should report a partition of -1
readerStatsPartition := config.Partition
if config.GroupID != "" {
readerStatsPartition = -1
}
// when configured as a consume group, start version as 1 to ensure that only
// the rebalance function will start readers
version := int64(0)
if config.GroupID != "" {
version = 1
}
stctx, stop := context.WithCancel(context.Background())
r := &Reader{
config: config,
msgs: make(chan readerMessage, config.QueueCapacity),
cancel: func() {},
commits: make(chan commitRequest, config.QueueCapacity),
stop: stop,
offset: FirstOffset,
stctx: stctx,
stats: &readerStats{
dialTime: makeSummary(),
readTime: makeSummary(),
waitTime: makeSummary(),
fetchSize: makeSummary(),
fetchBytes: makeSummary(),
// Generate the string representation of the partition number only
// once when the reader is created.
partition: strconv.Itoa(readerStatsPartition),
},
version: version,
}
if r.useConsumerGroup() {
r.done = make(chan struct{})
r.runError = make(chan error)
cg, err := NewConsumerGroup(ConsumerGroupConfig{
ID: r.config.GroupID,
Brokers: r.config.Brokers,
Dialer: r.config.Dialer,
Topics: r.getTopics(),
GroupBalancers: r.config.GroupBalancers,
HeartbeatInterval: r.config.HeartbeatInterval,
PartitionWatchInterval: r.config.PartitionWatchInterval,
WatchPartitionChanges: r.config.WatchPartitionChanges,
SessionTimeout: r.config.SessionTimeout,
RebalanceTimeout: r.config.RebalanceTimeout,
JoinGroupBackoff: r.config.JoinGroupBackoff,
RetentionTime: r.config.RetentionTime,
StartOffset: r.config.StartOffset,
Logger: r.config.Logger,
ErrorLogger: r.config.ErrorLogger,
})
if err != nil {
panic(err)
}
go r.run(cg)
}
return r
}
// Config returns the reader's configuration.
func (r *Reader) Config() ReaderConfig {
return r.config
}
// Close closes the stream, preventing the program from reading any more
// messages from it.
func (r *Reader) Close() error {
atomic.StoreUint32(&r.once, 1)
r.mutex.Lock()
closed := r.closed
r.closed = true
r.mutex.Unlock()
r.cancel()
r.stop()
r.join.Wait()
if r.done != nil {
<-r.done
}
if !closed {
close(r.msgs)
}
return nil
}
// ReadMessage reads and return the next message from the r. The method call
// blocks until a message becomes available, or an error occurs. The program
// may also specify a context to asynchronously cancel the blocking operation.
//
// The method returns io.EOF to indicate that the reader has been closed.
//
// If consumer groups are used, ReadMessage will automatically commit the
// offset when called. Note that this could result in an offset being committed
// before the message is fully processed.
//
// If more fine grained control of when offsets are committed is required, it
// is recommended to use FetchMessage with CommitMessages instead.
func (r *Reader) ReadMessage(ctx context.Context) (Message, error) {
m, err := r.FetchMessage(ctx)
if err != nil {
return Message{}, err
}
if r.useConsumerGroup() {
if err := r.CommitMessages(ctx, m); err != nil {
return Message{}, err
}
}
return m, nil
}
// FetchMessage reads and return the next message from the r. The method call
// blocks until a message becomes available, or an error occurs. The program
// may also specify a context to asynchronously cancel the blocking operation.
//
// The method returns io.EOF to indicate that the reader has been closed.
//
// FetchMessage does not commit offsets automatically when using consumer groups.
// Use CommitMessages to commit the offset.
func (r *Reader) FetchMessage(ctx context.Context) (Message, error) {
r.activateReadLag()
for {
r.mutex.Lock()
if !r.closed && r.version == 0 {
r.start(r.getTopicPartitionOffset())
}
version := r.version
r.mutex.Unlock()
select {
case <-ctx.Done():
return Message{}, ctx.Err()
case err := <-r.runError:
return Message{}, err
case m, ok := <-r.msgs:
if !ok {
return Message{}, io.EOF
}
if m.version >= version {
r.mutex.Lock()
switch {
case m.error != nil:
case version == r.version:
r.offset = m.message.Offset + 1
r.lag = m.watermark - r.offset
}
r.mutex.Unlock()
if errors.Is(m.error, io.EOF) {
// io.EOF is used as a marker to indicate that the stream
// has been closed, in case it was received from the inner
// reader we don't want to confuse the program and replace
// the error with io.ErrUnexpectedEOF.
m.error = io.ErrUnexpectedEOF
}
return m.message, m.error
}
}
}
}
// CommitMessages commits the list of messages passed as argument. The program
// may pass a context to asynchronously cancel the commit operation when it was
// configured to be blocking.
//
// Because kafka consumer groups track a single offset per partition, the
// highest message offset passed to CommitMessages will cause all previous
// messages to be committed. Applications need to account for these Kafka
// limitations when committing messages, and maintain message ordering if they
// need strong delivery guarantees. This property makes it valid to pass only
// the last message seen to CommitMessages in order to move the offset of the
// topic/partition it belonged to forward, effectively committing all previous
// messages in the partition.
func (r *Reader) CommitMessages(ctx context.Context, msgs ...Message) error {
if !r.useConsumerGroup() {
return errOnlyAvailableWithGroup
}
var errch <-chan error
creq := commitRequest{
commits: makeCommits(msgs...),
}
if r.useSyncCommits() {
ch := make(chan error, 1)
errch, creq.errch = ch, ch
}
select {
case r.commits <- creq:
case <-ctx.Done():
return ctx.Err()
case <-r.stctx.Done():
// This context is used to ensure we don't allow commits after the
// reader was closed.
return io.ErrClosedPipe
}
if !r.useSyncCommits() {
return nil
}
select {
case <-ctx.Done():
return ctx.Err()
case err := <-errch:
return err
}
}
// ReadLag returns the current lag of the reader by fetching the last offset of
// the topic and partition and computing the difference between that value and
// the offset of the last message returned by ReadMessage.
//
// This method is intended to be used in cases where a program may be unable to
// call ReadMessage to update the value returned by Lag, but still needs to get
// an up to date estimation of how far behind the reader is. For example when
// the consumer is not ready to process the next message.
//
// The function returns a lag of zero when the reader's current offset is
// negative.
func (r *Reader) ReadLag(ctx context.Context) (lag int64, err error) {
if r.useConsumerGroup() {
return 0, errNotAvailableWithGroup
}
type offsets struct {
first int64
last int64
}
offch := make(chan offsets, 1)
errch := make(chan error, 1)
go func() {
var off offsets
var err error
for _, broker := range r.config.Brokers {
var conn *Conn
if conn, err = r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition); err != nil {
continue
}
deadline, _ := ctx.Deadline()
conn.SetDeadline(deadline)
off.first, off.last, err = conn.ReadOffsets()
conn.Close()
if err == nil {
break
}
}
if err != nil {
errch <- err
} else {
offch <- off
}
}()
select {
case off := <-offch:
switch cur := r.Offset(); {
case cur == FirstOffset:
lag = off.last - off.first
case cur == LastOffset:
lag = 0
default:
lag = off.last - cur
}
case err = <-errch:
case <-ctx.Done():
err = ctx.Err()
}
return
}
// Offset returns the current absolute offset of the reader, or -1
// if r is backed by a consumer group.
func (r *Reader) Offset() int64 {
if r.useConsumerGroup() {
return -1
}
r.mutex.Lock()
offset := r.offset
r.mutex.Unlock()
r.withLogger(func(log Logger) {
log.Printf("looking up offset of kafka reader for partition %d of %s: %s", r.config.Partition, r.config.Topic, toHumanOffset(offset))
})
return offset
}
// Lag returns the lag of the last message returned by ReadMessage, or -1
// if r is backed by a consumer group.
func (r *Reader) Lag() int64 {
if r.useConsumerGroup() {
return -1
}
r.mutex.Lock()