forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
reader_test.go
1315 lines (1141 loc) · 31.9 KB
/
reader_test.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"
"io"
"math/rand"
"reflect"
"strconv"
"sync"
"testing"
"time"
)
func TestReader(t *testing.T) {
t.Parallel()
tests := []struct {
scenario string
function func(*testing.T, context.Context, *Reader)
}{
{
scenario: "calling Read with a context that has been canceled returns an error",
function: testReaderReadCanceled,
},
{
scenario: "all messages of the stream are returned when calling ReadMessage repeatedly",
function: testReaderReadMessages,
},
{
scenario: "test special offsets -1 and -2",
function: testReaderSetSpecialOffsets,
},
{
scenario: "setting the offset to random values returns the expected messages when Read is called",
function: testReaderSetRandomOffset,
},
{
scenario: "setting the offset by TimeStamp",
function: testReaderSetOffsetAt,
},
{
scenario: "calling Lag returns the lag of the last message read from kafka",
function: testReaderLag,
},
{
scenario: "calling ReadLag returns the current lag of a reader",
function: testReaderReadLag,
},
{ // https://github.com/segmentio/kafka-go/issues/30
scenario: "reading from an out-of-range offset waits until the context is cancelled",
function: testReaderOutOfRangeGetsCanceled,
},
}
for _, test := range tests {
testFunc := test.function
t.Run(test.scenario, func(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
r := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: makeTopic(),
MinBytes: 1,
MaxBytes: 10e6,
MaxWait: 100 * time.Millisecond,
})
defer r.Close()
testFunc(t, ctx, r)
})
}
}
func testReaderReadCanceled(t *testing.T, ctx context.Context, r *Reader) {
ctx, cancel := context.WithCancel(ctx)
cancel()
if _, err := r.ReadMessage(ctx); err != context.Canceled {
t.Error(err)
}
}
func testReaderReadMessages(t *testing.T, ctx context.Context, r *Reader) {
const N = 1000
prepareReader(t, ctx, r, makeTestSequence(N)...)
var offset int64
for i := 0; i != N; i++ {
m, err := r.ReadMessage(ctx)
if err != nil {
t.Error("reading message at offset", offset, "failed:", err)
return
}
offset = m.Offset + 1
v, _ := strconv.Atoi(string(m.Value))
if v != i {
t.Error("message at index", i, "has wrong value:", v)
return
}
}
}
func testReaderSetSpecialOffsets(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, ctx, r, Message{Value: []byte("first")})
prepareReader(t, ctx, r, makeTestSequence(3)...)
go func() {
time.Sleep(1 * time.Second)
prepareReader(t, ctx, r, Message{Value: []byte("last")})
}()
for _, test := range []struct {
off, final int64
want string
}{
{FirstOffset, 1, "first"},
{LastOffset, 5, "last"},
} {
offset := test.off
if err := r.SetOffset(offset); err != nil {
t.Error("setting offset", offset, "failed:", err)
}
m, err := r.ReadMessage(ctx)
if err != nil {
t.Error("reading at offset", offset, "failed:", err)
}
if string(m.Value) != test.want {
t.Error("message at offset", offset, "has wrong value:", string(m.Value))
}
if off := r.Offset(); off != test.final {
t.Errorf("bad final offset: got %d, want %d", off, test.final)
}
}
}
func testReaderSetRandomOffset(t *testing.T, ctx context.Context, r *Reader) {
const N = 10
prepareReader(t, ctx, r, makeTestSequence(N)...)
for i := 0; i != 2*N; i++ {
offset := rand.Intn(N)
r.SetOffset(int64(offset))
m, err := r.ReadMessage(ctx)
if err != nil {
t.Error("seeking to offset", offset, "failed:", err)
return
}
v, _ := strconv.Atoi(string(m.Value))
if v != offset {
t.Error("message at offset", offset, "has wrong value:", v)
return
}
}
}
func testReaderSetOffsetAt(t *testing.T, ctx context.Context, r *Reader) {
// We make 2 batches of messages here with a brief 2 second pause
// to ensure messages 0...9 will be written a few seconds before messages 10...19
// We'll then fetch the timestamp for message offset 10 and use that timestamp to set
// our reader
const N = 10
prepareReader(t, ctx, r, makeTestSequence(N)...)
time.Sleep(time.Second * 2)
prepareReader(t, ctx, r, makeTestSequence(N)...)
var ts time.Time
for i := 0; i < N*2; i++ {
m, err := r.ReadMessage(ctx)
if err != nil {
t.Error("error reading message", err)
}
// grab the time for the 10th message
if i == 10 {
ts = m.Time
}
}
err := r.SetOffsetAt(ctx, ts)
if err != nil {
t.Fatal("error setting offset by timestamp", err)
}
m, err := r.ReadMessage(context.Background())
if err != nil {
t.Fatal("error reading message", err)
}
if m.Offset != 10 {
t.Errorf("expected offset of 10, received offset %d", m.Offset)
}
}
func testReaderLag(t *testing.T, ctx context.Context, r *Reader) {
const N = 5
prepareReader(t, ctx, r, makeTestSequence(N)...)
if lag := r.Lag(); lag != 0 {
t.Errorf("the initial lag value is %d but was expected to be 0", lag)
}
for i := 0; i != N; i++ {
r.ReadMessage(ctx)
expect := int64(N - (i + 1))
if lag := r.Lag(); lag != expect {
t.Errorf("the lag value at offset %d is %d but was expected to be %d", i, lag, expect)
}
}
}
func testReaderReadLag(t *testing.T, ctx context.Context, r *Reader) {
const N = 5
prepareReader(t, ctx, r, makeTestSequence(N)...)
if lag, err := r.ReadLag(ctx); err != nil {
t.Error(err)
} else if lag != N {
t.Errorf("the initial lag value is %d but was expected to be %d", lag, N)
}
for i := 0; i != N; i++ {
r.ReadMessage(ctx)
expect := int64(N - (i + 1))
if lag, err := r.ReadLag(ctx); err != nil {
t.Error(err)
} else if lag != expect {
t.Errorf("the lag value at offset %d is %d but was expected to be %d", i, lag, expect)
}
}
}
func testReaderOutOfRangeGetsCanceled(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, ctx, r, makeTestSequence(10)...)
const D = 100 * time.Millisecond
t0 := time.Now()
ctx, cancel := context.WithTimeout(ctx, D)
defer cancel()
if err := r.SetOffset(42); err != nil {
t.Error(err)
}
_, err := r.ReadMessage(ctx)
if err != context.DeadlineExceeded {
t.Error("bad error:", err)
}
t1 := time.Now()
if d := t1.Sub(t0); d < D {
t.Error("ReadMessage returned too early after", d)
}
}
func createTopic(t *testing.T, topic string, partitions int) {
conn, err := Dial("tcp", "localhost:9092")
if err != nil {
t.Error("bad conn")
return
}
defer conn.Close()
_, err = conn.createTopics(createTopicsRequestV0{
Topics: []createTopicsRequestV0Topic{
{
Topic: topic,
NumPartitions: int32(partitions),
ReplicationFactor: 1,
},
},
Timeout: int32(30 * time.Second / time.Millisecond),
})
switch err {
case nil:
// ok
case TopicAlreadyExists:
// ok
default:
t.Error("bad createTopics", err)
t.FailNow()
}
}
func TestReaderOnNonZeroPartition(t *testing.T) {
t.Parallel()
tests := []struct {
scenario string
function func(*testing.T, context.Context, *Reader)
}{
{
scenario: "topic and partition should now be included in header",
function: testReaderSetsTopicAndPartition,
},
}
for _, test := range tests {
testFunc := test.function
t.Run(test.scenario, func(t *testing.T) {
t.Parallel()
topic := makeTopic()
createTopic(t, topic, 2)
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
r := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic,
Partition: 1,
MinBytes: 1,
MaxBytes: 10e6,
MaxWait: 100 * time.Millisecond,
})
defer r.Close()
testFunc(t, ctx, r)
})
}
}
func testReaderSetsTopicAndPartition(t *testing.T, ctx context.Context, r *Reader) {
const N = 3
prepareReader(t, ctx, r, makeTestSequence(N)...)
for i := 0; i != N; i++ {
m, err := r.ReadMessage(ctx)
if err != nil {
t.Error("reading message failed:", err)
return
}
if m.Topic == "" {
t.Error("expected topic to be set")
return
}
if m.Topic != r.config.Topic {
t.Errorf("expected message to contain topic, %v; got %v", r.config.Topic, m.Topic)
return
}
if m.Partition != r.config.Partition {
t.Errorf("expected partition to be set; expected 1, got %v", m.Partition)
return
}
}
}
// TestReadTruncatedMessages uses a configuration designed to get the Broker to
// return truncated messages. It exercises the case where an earlier bug caused
// reading to time out by attempting to read beyond the current response. This
// test is not perfect, but it is pretty reliable about reproducing the issue.
//
// NOTE : it currently only succeeds against kafka 0.10.1.0, so it will be
// skipped. It's here so that it can be manually run.
func TestReadTruncatedMessages(t *testing.T) {
// todo : it would be great to get it to work against 0.11.0.0 so we could
// include it in CI unit tests.
t.Skip()
t.Parallel()
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
r := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: makeTopic(),
MinBytes: 1,
MaxBytes: 100,
MaxWait: 100 * time.Millisecond,
})
defer r.Close()
n := 500
prepareReader(t, ctx, r, makeTestSequence(n)...)
for i := 0; i < n; i++ {
if _, err := r.ReadMessage(ctx); err != nil {
t.Fatal(err)
}
}
}
func makeTestSequence(n int) []Message {
base := time.Now()
msgs := make([]Message, n)
for i := 0; i != n; i++ {
msgs[i] = Message{
Time: base.Add(time.Duration(i) * time.Millisecond).Truncate(time.Millisecond),
Value: []byte(strconv.Itoa(i)),
}
}
return msgs
}
func prepareReader(t *testing.T, ctx context.Context, r *Reader, msgs ...Message) {
var config = r.Config()
var conn *Conn
var err error
for {
if conn, err = DialLeader(ctx, "tcp", "localhost:9092", config.Topic, config.Partition); err == nil {
break
}
select {
case <-time.After(time.Second):
case <-ctx.Done():
t.Fatal(ctx.Err())
}
}
defer conn.Close()
if _, err := conn.WriteMessages(msgs...); err != nil {
t.Fatal(err)
}
}
var (
benchmarkReaderOnce sync.Once
benchmarkReaderTopic = makeTopic()
benchmarkReaderPayload = make([]byte, 2*1024)
)
func BenchmarkReader(b *testing.B) {
const broker = "localhost:9092"
ctx := context.Background()
benchmarkReaderOnce.Do(func() {
conn, err := DialLeader(ctx, "tcp", broker, benchmarkReaderTopic, 0)
if err != nil {
b.Fatal(err)
}
defer conn.Close()
msgs := make([]Message, 1000)
for i := range msgs {
msgs[i].Value = benchmarkReaderPayload
}
for i := 0; i != 10; i++ { // put 10K messages
if _, err := conn.WriteMessages(msgs...); err != nil {
b.Fatal(err)
}
}
b.ResetTimer()
})
r := NewReader(ReaderConfig{
Brokers: []string{broker},
Topic: benchmarkReaderTopic,
Partition: 0,
MinBytes: 1e3,
MaxBytes: 1e6,
MaxWait: 100 * time.Millisecond,
})
for i := 0; i < b.N; i++ {
if (i % 10000) == 0 {
r.SetOffset(-1)
}
_, err := r.ReadMessage(ctx)
if err != nil {
b.Fatal(err)
}
}
r.Close()
b.SetBytes(int64(len(benchmarkReaderPayload)))
}
func TestCloseLeavesGroup(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
topic := makeTopic()
createTopic(t, topic, 1)
groupID := makeGroupID()
r := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic,
GroupID: groupID,
MinBytes: 1,
MaxBytes: 10e6,
MaxWait: 100 * time.Millisecond,
RebalanceTimeout: time.Second,
})
prepareReader(t, ctx, r, Message{Value: []byte("test")})
conn, err := Dial("tcp", r.config.Brokers[0])
if err != nil {
t.Fatalf("error dialing: %v", err)
}
defer conn.Close()
descGroups := func() describeGroupsResponseV0 {
resp, err := conn.describeGroups(describeGroupsRequestV0{
GroupIDs: []string{groupID},
})
if err != nil {
t.Fatalf("error from describeGroups %v", err)
}
return resp
}
_, err = r.ReadMessage(ctx)
if err != nil {
t.Fatalf("our reader never joind its group or couldn't read a message: %v", err)
}
resp := descGroups()
if len(resp.Groups) != 1 {
t.Fatalf("expected 1 group. got: %d", len(resp.Groups))
}
if len(resp.Groups[0].Members) != 1 {
t.Fatalf("expected group membership size of %d, but got %d", 1, len(resp.Groups[0].Members))
}
err = r.Close()
if err != nil {
t.Fatalf("unexpected error closing reader: %s", err.Error())
}
resp = descGroups()
if len(resp.Groups) != 1 {
t.Fatalf("expected 1 group. got: %d", len(resp.Groups))
}
if len(resp.Groups[0].Members) != 0 {
t.Fatalf("expected group membership size of %d, but got %d", 0, len(resp.Groups[0].Members))
}
}
func testConsumerGroupImmediateClose(t *testing.T, ctx context.Context, r *Reader) {
if err := r.Close(); err != nil {
t.Fatalf("bad err: %v", err)
}
}
func testConsumerGroupSimple(t *testing.T, ctx context.Context, r *Reader) {
if err := r.Close(); err != nil {
t.Fatalf("bad err: %v", err)
}
}
func TestReaderSetOffsetWhenConsumerGroupsEnabled(t *testing.T) {
r := &Reader{config: ReaderConfig{GroupID: "not-zero"}}
if err := r.SetOffset(LastOffset); err != errNotAvailableWithGroup {
t.Fatalf("expected %v; got %v", errNotAvailableWithGroup, err)
}
}
func TestReaderOffsetWhenConsumerGroupsEnabled(t *testing.T) {
r := &Reader{config: ReaderConfig{GroupID: "not-zero"}}
if offset := r.Offset(); offset != -1 {
t.Fatalf("expected -1; got %v", offset)
}
}
func TestReaderLagWhenConsumerGroupsEnabled(t *testing.T) {
r := &Reader{config: ReaderConfig{GroupID: "not-zero"}}
if offset := r.Lag(); offset != -1 {
t.Fatalf("expected -1; got %v", offset)
}
}
func TestReaderPartitionWhenConsumerGroupsEnabled(t *testing.T) {
invoke := func() (boom bool) {
defer func() {
if r := recover(); r != nil {
boom = true
}
}()
NewReader(ReaderConfig{
GroupID: "set",
Partition: 1,
})
return false
}
if !invoke() {
t.Fatalf("expected panic; but NewReader worked?!")
}
}
func TestExtractTopics(t *testing.T) {
testCases := map[string]struct {
Members []GroupMember
Topics []string
}{
"nil": {},
"single member, single topic": {
Members: []GroupMember{
{
ID: "a",
Topics: []string{"topic"},
},
},
Topics: []string{"topic"},
},
"two members, single topic": {
Members: []GroupMember{
{
ID: "a",
Topics: []string{"topic"},
},
{
ID: "b",
Topics: []string{"topic"},
},
},
Topics: []string{"topic"},
},
"two members, two topics": {
Members: []GroupMember{
{
ID: "a",
Topics: []string{"topic-1"},
},
{
ID: "b",
Topics: []string{"topic-2"},
},
},
Topics: []string{"topic-1", "topic-2"},
},
"three members, three shared topics": {
Members: []GroupMember{
{
ID: "a",
Topics: []string{"topic-1", "topic-2"},
},
{
ID: "b",
Topics: []string{"topic-2", "topic-3"},
},
{
ID: "c",
Topics: []string{"topic-3", "topic-1"},
},
},
Topics: []string{"topic-1", "topic-2", "topic-3"},
},
}
for label, tc := range testCases {
t.Run(label, func(t *testing.T) {
topics := extractTopics(tc.Members)
if !reflect.DeepEqual(tc.Topics, topics) {
t.Errorf("expected %v; got %v", tc.Topics, topics)
}
})
}
}
func TestReaderConsumerGroup(t *testing.T) {
t.Parallel()
tests := []struct {
scenario string
partitions int
commitInterval time.Duration
function func(*testing.T, context.Context, *Reader)
}{
{
scenario: "basic handshake",
partitions: 1,
function: testReaderConsumerGroupHandshake,
},
{
scenario: "verify offset committed",
partitions: 1,
function: testReaderConsumerGroupVerifyOffsetCommitted,
},
{
scenario: "verify offset committed when using interval committer",
partitions: 1,
commitInterval: 400 * time.Millisecond,
function: testReaderConsumerGroupVerifyPeriodicOffsetCommitter,
},
{
scenario: "rebalance across many partitions and consumers",
partitions: 8,
function: testReaderConsumerGroupRebalanceAcrossManyPartitionsAndConsumers,
},
{
scenario: "consumer group commits on close",
partitions: 3,
function: testReaderConsumerGroupVerifyCommitsOnClose,
},
{
scenario: "consumer group rebalance",
partitions: 3,
function: testReaderConsumerGroupRebalance,
},
{
scenario: "consumer group rebalance across topics",
partitions: 3,
function: testReaderConsumerGroupRebalanceAcrossTopics,
},
{
scenario: "consumer group reads content across partitions",
partitions: 3,
function: testReaderConsumerGroupReadContentAcrossPartitions,
},
{
scenario: "Close immediately after NewReader",
partitions: 1,
function: testConsumerGroupImmediateClose,
},
{
scenario: "Close immediately after NewReader",
partitions: 1,
function: testConsumerGroupSimple,
},
}
for _, test := range tests {
t.Run(test.scenario, func(t *testing.T) {
t.Parallel()
topic := makeTopic()
createTopic(t, topic, test.partitions)
groupID := makeGroupID()
r := NewReader(ReaderConfig{
Brokers: []string{"localhost:9092"},
Topic: topic,
GroupID: groupID,
HeartbeatInterval: 2 * time.Second,
CommitInterval: test.commitInterval,
RebalanceTimeout: 2 * time.Second,
RetentionTime: time.Hour,
MinBytes: 1,
MaxBytes: 1e6,
})
defer r.Close()
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
test.function(t, ctx, r)
})
}
}
func testReaderConsumerGroupHandshake(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, context.Background(), r, makeTestSequence(5)...)
m, err := r.ReadMessage(ctx)
if err != nil {
t.Errorf("bad err: %v", err)
}
if m.Topic != r.config.Topic {
t.Errorf("topic not set")
}
if m.Offset != 0 {
t.Errorf("offset not set")
}
m, err = r.ReadMessage(ctx)
if err != nil {
t.Errorf("bad err: %v", err)
}
if m.Topic != r.config.Topic {
t.Errorf("topic not set")
}
if m.Offset != 1 {
t.Errorf("offset not set")
}
}
func testReaderConsumerGroupVerifyOffsetCommitted(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, context.Background(), r, makeTestSequence(3)...)
if _, err := r.FetchMessage(ctx); err != nil {
t.Errorf("bad err: %v", err) // skip the first message
}
m, err := r.FetchMessage(ctx)
if err != nil {
t.Errorf("bad err: %v", err)
}
if err := r.CommitMessages(ctx, m); err != nil {
t.Errorf("bad commit message: %v", err)
}
offsets := getOffsets(t, r.config)
if expected := map[int]int64{0: m.Offset + 1}; !reflect.DeepEqual(expected, offsets) {
t.Errorf("expected %v; got %v", expected, offsets)
}
}
func testReaderConsumerGroupVerifyPeriodicOffsetCommitter(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, context.Background(), r, makeTestSequence(3)...)
if _, err := r.FetchMessage(ctx); err != nil {
t.Errorf("bad err: %v", err) // skip the first message
}
m, err := r.FetchMessage(ctx)
if err != nil {
t.Errorf("bad err: %v", err)
}
started := time.Now()
if err := r.CommitMessages(ctx, m); err != nil {
t.Errorf("bad commit message: %v", err)
}
if elapsed := time.Now().Sub(started); elapsed > 10*time.Millisecond {
t.Errorf("background commits should happen nearly instantly")
}
// wait for committer to pick up the commits
time.Sleep(r.config.CommitInterval * 3)
offsets := getOffsets(t, r.config)
if expected := map[int]int64{0: m.Offset + 1}; !reflect.DeepEqual(expected, offsets) {
t.Errorf("expected %v; got %v", expected, offsets)
}
}
func testReaderConsumerGroupVerifyCommitsOnClose(t *testing.T, ctx context.Context, r *Reader) {
prepareReader(t, context.Background(), r, makeTestSequence(3)...)
if _, err := r.FetchMessage(ctx); err != nil {
t.Errorf("bad err: %v", err) // skip the first message
}
m, err := r.FetchMessage(ctx)
if err != nil {
t.Errorf("bad err: %v", err)
}
if err := r.CommitMessages(ctx, m); err != nil {
t.Errorf("bad commit message: %v", err)
}
if err := r.Close(); err != nil {
t.Errorf("bad Close: %v", err)
}
r2 := NewReader(r.config)
defer r2.Close()
offsets := getOffsets(t, r2.config)
if expected := map[int]int64{0: m.Offset + 1}; !reflect.DeepEqual(expected, offsets) {
t.Errorf("expected %v; got %v", expected, offsets)
}
}
func testReaderConsumerGroupReadContentAcrossPartitions(t *testing.T, ctx context.Context, r *Reader) {
const N = 12
writer := NewWriter(WriterConfig{
Brokers: r.config.Brokers,
Topic: r.config.Topic,
Dialer: r.config.Dialer,
Balancer: &RoundRobin{},
BatchSize: 1,
})
if err := writer.WriteMessages(ctx, makeTestSequence(N)...); err != nil {
t.Fatalf("bad write messages: %v", err)
}
if err := writer.Close(); err != nil {
t.Fatalf("bad write err: %v", err)
}
partitions := map[int]struct{}{}
for i := 0; i < N; i++ {
m, err := r.FetchMessage(ctx)
if err != nil {
t.Errorf("bad error: %s", err)
}
partitions[m.Partition] = struct{}{}
}
if v := len(partitions); v != 3 {
t.Errorf("expected messages across 3 partitions; got messages across %v partitions", v)
}
}
func testReaderConsumerGroupRebalance(t *testing.T, ctx context.Context, r *Reader) {
r2 := NewReader(r.config)
defer r.Close()
const (
N = 12
partitions = 2
)
// rebalance should result in 12 message in each of the partitions
writer := NewWriter(WriterConfig{
Brokers: r.config.Brokers,
Topic: r.config.Topic,
Dialer: r.config.Dialer,
Balancer: &RoundRobin{},
BatchSize: 1,
})
if err := writer.WriteMessages(ctx, makeTestSequence(N*partitions)...); err != nil {
t.Fatalf("bad write messages: %v", err)
}
if err := writer.Close(); err != nil {
t.Fatalf("bad write err: %v", err)
}
// after rebalance, each reader should have a partition to itself
for i := 0; i < N; i++ {
if _, err := r2.FetchMessage(ctx); err != nil {
t.Errorf("expect to read from reader 2")
}
if _, err := r.FetchMessage(ctx); err != nil {
t.Errorf("expect to read from reader 1")
}
}
}
func testReaderConsumerGroupRebalanceAcrossTopics(t *testing.T, ctx context.Context, r *Reader) {
// create a second reader that shares the groupID, but reads from a different topic
topic2 := makeTopic()
createTopic(t, topic2, 1)
r2 := NewReader(ReaderConfig{
Brokers: r.config.Brokers,
Topic: topic2,
GroupID: r.config.GroupID,
HeartbeatInterval: r.config.HeartbeatInterval,
SessionTimeout: r.config.SessionTimeout,
RetentionTime: r.config.RetentionTime,
MinBytes: r.config.MinBytes,
MaxBytes: r.config.MaxBytes,
Logger: r.config.Logger,
})
defer r.Close()
prepareReader(t, ctx, r2, makeTestSequence(1)...)
const (
N = 12
)
// write messages across both partitions
writer := NewWriter(WriterConfig{
Brokers: r.config.Brokers,
Topic: r.config.Topic,
Dialer: r.config.Dialer,
Balancer: &RoundRobin{},
BatchSize: 1,
})
if err := writer.WriteMessages(ctx, makeTestSequence(N)...); err != nil {
t.Fatalf("bad write messages: %v", err)
}
if err := writer.Close(); err != nil {
t.Fatalf("bad write err: %v", err)
}
// after rebalance, r2 should read topic2 and r1 should read ALL of the original topic
if _, err := r2.FetchMessage(ctx); err != nil {
t.Errorf("expect to read from reader 2")
}
// all N messages on the original topic should be read by the original reader
for i := 0; i < N; i++ {
if _, err := r.FetchMessage(ctx); err != nil {
t.Errorf("expect to read from reader 1")
}
}
}
func testReaderConsumerGroupRebalanceAcrossManyPartitionsAndConsumers(t *testing.T, ctx context.Context, r *Reader) {
// I've rebalanced up to 100 servers, but the rebalance can take upwards
// of a minute and that seems too long for unit tests. Also, setting this
// to a larger number seems to make the kafka broker unresponsive.
// TODO research if there's a way to reduce rebalance time across many partitions
// svls: the described behavior is due to the thundering herd of readers
// hitting the rebalance timeout. introducing the 100ms sleep in the
// loop below in order to give time for the sync group to finish has
// greatly helped, though we still hit the timeout from time to time.
const N = 8
var readers []*Reader
for i := 0; i < N-1; i++ {
reader := NewReader(r.config)