-
Notifications
You must be signed in to change notification settings - Fork 792
/
conn_test.go
1371 lines (1155 loc) · 31 KB
/
conn_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 (
"bytes"
"context"
"errors"
"fmt"
"io"
"math/rand"
"net"
"os"
"strconv"
"testing"
"time"
ktesting "github.com/segmentio/kafka-go/testing"
"golang.org/x/net/nettest"
)
type timeout struct{}
func (*timeout) Error() string { return "timeout" }
func (*timeout) Temporary() bool { return true }
func (*timeout) Timeout() bool { return true }
// connPipe is an adapter that implements the net.Conn interface on top of
// two client kafka connections to pass the nettest.TestConn test suite.
type connPipe struct {
rconn *Conn
wconn *Conn
}
func (c *connPipe) Close() error {
b := [1]byte{} // marker that the connection has been closed
c.wconn.SetWriteDeadline(time.Time{})
c.wconn.Write(b[:])
c.wconn.Close()
c.rconn.Close()
return nil
}
func (c *connPipe) Read(b []byte) (int, error) {
// See comments in Write.
time.Sleep(time.Millisecond)
if t := c.rconn.readDeadline(); !t.IsZero() {
return 0, &timeout{}
}
n, err := c.rconn.Read(b)
if n == 1 && b[0] == 0 {
c.rconn.Close()
n, err = 0, io.EOF
}
return n, err
}
func (c *connPipe) Write(b []byte) (int, error) {
// The nettest/ConcurrentMethods test spawns a bunch of goroutines that do
// random stuff on the connection, if a Read or Write was issued before a
// deadline was set then it could cancel an inflight request to kafka,
// resulting in the connection being closed.
// To prevent this from happening we wait a little while to give the other
// goroutines a chance to start and set the deadline.
time.Sleep(time.Millisecond)
// The nettest code only sets deadlines when it expects the write to time
// out. The broker connection is alive and able to accept data, so we need
// to simulate the timeout in order to get the tests to pass.
if t := c.wconn.writeDeadline(); !t.IsZero() {
return 0, &timeout{}
}
return c.wconn.Write(b)
}
func (c *connPipe) LocalAddr() net.Addr {
return c.rconn.LocalAddr()
}
func (c *connPipe) RemoteAddr() net.Addr {
return c.wconn.LocalAddr()
}
func (c *connPipe) SetDeadline(t time.Time) error {
c.rconn.SetDeadline(t)
c.wconn.SetDeadline(t)
return nil
}
func (c *connPipe) SetReadDeadline(t time.Time) error {
return c.rconn.SetReadDeadline(t)
}
func (c *connPipe) SetWriteDeadline(t time.Time) error {
return c.wconn.SetWriteDeadline(t)
}
func init() {
rand.Seed(time.Now().UnixNano())
}
func makeTopic() string {
return fmt.Sprintf("kafka-go-%016x", rand.Int63())
}
func makeGroupID() string {
return fmt.Sprintf("kafka-go-group-%016x", rand.Int63())
}
func makeTransactionalID() string {
return fmt.Sprintf("kafka-go-transactional-id-%016x", rand.Int63())
}
func TestConn(t *testing.T) {
tests := []struct {
scenario string
function func(*testing.T, *Conn)
minVersion string
}{
{
scenario: "close right away",
function: testConnClose,
},
{
scenario: "ensure the initial offset of a connection is the first offset",
function: testConnFirstOffset,
},
{
scenario: "write a single message to kafka should succeed",
function: testConnWrite,
},
{
scenario: "writing a message to a closed kafka connection should fail",
function: testConnCloseAndWrite,
},
{
scenario: "ensure the connection can seek to the first offset",
function: testConnSeekFirstOffset,
},
{
scenario: "ensure the connection can seek to the last offset",
function: testConnSeekLastOffset,
},
{
scenario: "ensure the connection can seek relative to the current offset",
function: testConnSeekCurrentOffset,
},
{
scenario: "ensure the connection can seek to a random offset",
function: testConnSeekRandomOffset,
},
{
scenario: "unchecked seeks allow the connection to be positioned outside the boundaries of the partition",
function: testConnSeekDontCheck,
},
{
scenario: "writing and reading messages sequentially should preserve the order",
function: testConnWriteReadSequentially,
},
{
scenario: "writing a batch of messages and reading it sequentially should preserve the order",
function: testConnWriteBatchReadSequentially,
},
{
scenario: "writing and reading messages concurrently should preserve the order",
function: testConnWriteReadConcurrently,
},
{
scenario: "reading messages with a buffer that is too short should return io.ErrShortBuffer and maintain the connection open",
function: testConnReadShortBuffer,
},
{
scenario: "reading messages from an empty partition should timeout after reaching the deadline",
function: testConnReadEmptyWithDeadline,
},
{
scenario: "write batch of messages and read the highest offset (watermark)",
function: testConnReadWatermarkFromBatch,
},
{
scenario: "read a batch with no explicit min or max bytes",
function: testConnReadBatchWithNoMinMaxBytes,
minVersion: "0.11.0",
},
{
scenario: "read a batch using explicit max wait time",
function: testConnReadBatchWithMaxWait,
},
{
scenario: "find the group coordinator",
function: testConnFindCoordinator,
},
{
scenario: "test join group with an invalid groupID",
function: testConnJoinGroupInvalidGroupID,
},
{
scenario: "test join group with an invalid sessionTimeout",
function: testConnJoinGroupInvalidSessionTimeout,
},
{
scenario: "test join group with an invalid refreshTimeout",
function: testConnJoinGroupInvalidRefreshTimeout,
},
{
scenario: "test heartbeat once group has been created",
function: testConnHeartbeatErr,
},
{
scenario: "test leave group returns error when called outside group",
function: testConnLeaveGroupErr,
},
{
scenario: "test sync group with bad memberID",
function: testConnSyncGroupErr,
},
{
scenario: "test list groups",
function: testConnListGroupsReturnsGroups,
minVersion: "0.11.0",
},
{
scenario: "test fetch and commit offset",
function: testConnFetchAndCommitOffsets,
},
{
scenario: "test delete topics",
function: testDeleteTopics,
},
{
scenario: "test delete topics with an invalid topic",
function: testDeleteTopicsInvalidTopic,
},
{
scenario: "test retrieve controller",
function: testController,
},
{
scenario: "test list brokers",
function: testBrokers,
},
{
scenario: "the connection advertises the broker that it is connected to",
function: testConnBroker,
},
}
const (
tcp = "tcp"
kafka = "localhost:9092"
)
for _, test := range tests {
if !ktesting.KafkaIsAtLeast(test.minVersion) {
t.Log("skipping " + test.scenario + " because broker is not at least version " + test.minVersion)
continue
}
testFunc := test.function
t.Run(test.scenario, func(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
topic := makeTopic()
conn, err := (&Dialer{
Resolver: &net.Resolver{},
}).DialLeader(ctx, tcp, kafka, topic, 0)
if err != nil {
t.Fatal("failed to open a new kafka connection:", err)
}
defer conn.Close()
testFunc(t, conn)
})
}
t.Run("nettest", func(t *testing.T) {
// Need ability to skip nettest on newer Kafka versions to avoid these kinds of errors:
// --- FAIL: TestConn/nettest (17.56s)
// --- FAIL: TestConn/nettest/PingPong (7.40s)
// conntest.go:112: unexpected Read error: [7] Request Timed Out: the request exceeded the user-specified time limit in the request
// conntest.go:118: mismatching value: got 77, want 78
// conntest.go:118: mismatching value: got 78, want 79
// ...
//
// TODO: Figure out why these are happening and fix them (they don't appear to be new).
if _, ok := os.LookupEnv("KAFKA_SKIP_NETTEST"); ok {
t.Log("skipping nettest because KAFKA_SKIP_NETTEST is set")
t.Skip()
}
t.Parallel()
nettest.TestConn(t, func() (c1 net.Conn, c2 net.Conn, stop func(), err error) {
topic1 := makeTopic()
topic2 := makeTopic()
var t1Reader *Conn
var t2Reader *Conn
var t1Writer *Conn
var t2Writer *Conn
dialer := &Dialer{}
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
if t1Reader, err = dialer.DialLeader(ctx, tcp, kafka, topic1, 0); err != nil {
return
}
if t2Reader, err = dialer.DialLeader(ctx, tcp, kafka, topic2, 0); err != nil {
return
}
if t1Writer, err = dialer.DialLeader(ctx, tcp, kafka, topic1, 0); err != nil {
return
}
if t2Writer, err = dialer.DialLeader(ctx, tcp, kafka, topic2, 0); err != nil {
return
}
stop = func() {
t1Reader.Close()
t1Writer.Close()
t2Reader.Close()
t2Writer.Close()
}
c1 = &connPipe{rconn: t1Reader, wconn: t2Writer}
c2 = &connPipe{rconn: t2Reader, wconn: t1Writer}
return
})
})
}
func testConnClose(t *testing.T, conn *Conn) {
if err := conn.Close(); err != nil {
t.Error(err)
}
}
func testConnFirstOffset(t *testing.T, conn *Conn) {
offset, whence := conn.Offset()
if offset != 0 && whence != 0 {
t.Error("bad first offset:", offset, whence)
}
}
func testConnWrite(t *testing.T, conn *Conn) {
b := []byte("Hello World!")
n, err := conn.Write(b)
if err != nil {
t.Error(err)
}
if n != len(b) {
t.Error("bad length returned by (*Conn).Write:", n)
}
}
func testConnCloseAndWrite(t *testing.T, conn *Conn) {
conn.Close()
_, err := conn.Write([]byte("Hello World!"))
// expect a network error
var netOpError *net.OpError
if !errors.As(err, &netOpError) {
t.Error(err)
}
}
func testConnSeekFirstOffset(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
offset, err := conn.Seek(0, SeekStart)
if err != nil {
t.Error(err)
}
if offset != 0 {
t.Error("bad offset:", offset)
}
}
func testConnSeekLastOffset(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
offset, err := conn.Seek(0, SeekEnd)
if err != nil {
t.Error(err)
}
if offset != 10 {
t.Error("bad offset:", offset)
}
}
func testConnSeekCurrentOffset(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
offset, err := conn.Seek(5, SeekStart)
if err != nil {
t.Error(err)
}
if offset != 5 {
t.Error("bad offset:", offset)
}
offset, err = conn.Seek(-2, SeekCurrent)
if err != nil {
t.Error(err)
}
if offset != 3 {
t.Error("bad offset:", offset)
}
}
func testConnSeekRandomOffset(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
offset, err := conn.Seek(3, SeekAbsolute)
if err != nil {
t.Error(err)
}
if offset != 3 {
t.Error("bad offset:", offset)
}
}
func testConnSeekDontCheck(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
offset, err := conn.Seek(42, SeekAbsolute|SeekDontCheck)
if err != nil {
t.Error(err)
}
if offset != 42 {
t.Error("bad offset:", offset)
}
if _, err := conn.ReadMessage(1024); !errors.Is(err, OffsetOutOfRange) {
t.Error("unexpected error:", err)
}
}
func testConnWriteReadSequentially(t *testing.T, conn *Conn) {
for i := 0; i != 10; i++ {
if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
t.Fatal(err)
}
}
b := make([]byte, 128)
for i := 0; i != 10; i++ {
n, err := conn.Read(b)
if err != nil {
t.Error(err)
continue
}
s := string(b[:n])
if v, err := strconv.Atoi(s); err != nil {
t.Error(err)
} else if v != i {
t.Errorf("bad message read at offset %d: %s", i, s)
}
}
}
func testConnWriteBatchReadSequentially(t *testing.T, conn *Conn) {
msgs := makeTestSequence(10)
if _, err := conn.WriteMessages(msgs...); err != nil {
t.Fatal(err)
}
for i := 0; i != 10; i++ {
msg, err := conn.ReadMessage(128)
if err != nil {
t.Error(err)
continue
}
if !bytes.Equal(msg.Key, msgs[i].Key) {
t.Errorf("bad message key at offset %d: %q != %q", i, msg.Key, msgs[i].Key)
}
if !bytes.Equal(msg.Value, msgs[i].Value) {
t.Errorf("bad message value at offset %d: %q != %q", i, msg.Value, msgs[i].Value)
}
if !msg.Time.Equal(msgs[i].Time) {
t.Errorf("bad message time at offset %d: %s != %s", i, msg.Time, msgs[i].Time)
}
}
}
func testConnReadWatermarkFromBatch(t *testing.T, conn *Conn) {
if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
t.Fatal(err)
}
const minBytes = 1
const maxBytes = 10e6 // 10 MB
value := make([]byte, 10e3) // 10 KB
batch := conn.ReadBatch(minBytes, maxBytes)
for i := 0; i < 10; i++ {
_, err := batch.Read(value)
if err != nil {
if err = batch.Close(); err != nil {
t.Fatalf("error trying to read batch message: %s", err)
}
}
if batch.HighWaterMark() != 10 {
t.Fatal("expected highest offset (watermark) to be 10")
}
}
batch.Close()
}
func testConnReadBatchWithNoMinMaxBytes(t *testing.T, conn *Conn) {
if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
t.Fatal(err)
}
value := make([]byte, 10e3) // 10 KB
batch := conn.ReadBatchWith(ReadBatchConfig{})
for i := 0; i < 10; i++ {
_, err := batch.Read(value)
if err != nil {
if err = batch.Close(); err != nil {
t.Fatalf("error trying to read batch message: %s", err)
}
}
if batch.HighWaterMark() != 10 {
t.Fatal("expected highest offset (watermark) to be 10")
}
}
if err := batch.Close(); err != nil {
t.Fatalf("error trying to close batch: %s", err)
}
if err := batch.Err(); err != nil {
t.Fatalf("broken batch: %s", err)
}
}
func testConnReadBatchWithMaxWait(t *testing.T, conn *Conn) {
if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
t.Fatal(err)
}
const maxBytes = 10e6 // 10 MB
value := make([]byte, 10e3) // 10 KB
cfg := ReadBatchConfig{
MinBytes: maxBytes, // use max for both so that we hit max wait time
MaxBytes: maxBytes,
MaxWait: 500 * time.Millisecond,
}
// set aa read deadline so the batch will succeed.
conn.SetDeadline(time.Now().Add(time.Second))
batch := conn.ReadBatchWith(cfg)
for i := 0; i < 10; i++ {
_, err := batch.Read(value)
if err != nil {
if err = batch.Close(); err != nil {
t.Fatalf("error trying to read batch message: %s", err)
}
}
if batch.HighWaterMark() != 10 {
t.Fatal("expected highest offset (watermark) to be 10")
}
}
batch.Close()
// reset the offset and ensure that the conn deadline takes precedence over
// the max wait
conn.Seek(0, SeekAbsolute)
conn.SetDeadline(time.Now().Add(50 * time.Millisecond))
batch = conn.ReadBatchWith(cfg)
var netErr net.Error
if err := batch.Err(); err == nil {
t.Fatal("should have timed out, but got no error")
} else if errors.As(err, &netErr) {
if !netErr.Timeout() {
t.Fatalf("should have timed out, but got: %v", err)
}
}
}
func waitForCoordinator(t *testing.T, conn *Conn, groupID string) {
// ensure that kafka has allocated a group coordinator. oddly, issue doesn't
// appear to happen if the kafka been running for a while.
const maxAttempts = 20
for attempt := 1; attempt <= maxAttempts; attempt++ {
_, err := conn.findCoordinator(findCoordinatorRequestV0{
CoordinatorKey: groupID,
})
if err != nil {
if errors.Is(err, GroupCoordinatorNotAvailable) {
time.Sleep(250 * time.Millisecond)
continue
} else {
t.Fatalf("unable to find coordinator for group: %v", err)
}
} else {
return
}
}
t.Fatalf("unable to connect to coordinator after %v attempts", maxAttempts)
}
func createGroup(t *testing.T, conn *Conn, groupID string) (generationID int32, memberID string, stop func()) {
waitForCoordinator(t, conn, groupID)
join := func() (joinGroup joinGroupResponseV1) {
var err error
for attempt := 0; attempt < 10; attempt++ {
joinGroup, err = conn.joinGroup(joinGroupRequestV1{
GroupID: groupID,
SessionTimeout: int32(time.Minute / time.Millisecond),
RebalanceTimeout: int32(time.Second / time.Millisecond),
ProtocolType: "roundrobin",
GroupProtocols: []joinGroupRequestGroupProtocolV1{
{
ProtocolName: "roundrobin",
ProtocolMetadata: []byte("blah"),
},
},
})
if err != nil {
if errors.Is(err, NotCoordinatorForGroup) {
time.Sleep(250 * time.Millisecond)
continue
} else {
t.Fatalf("bad joinGroup: %s", err)
}
} else {
return
}
}
return
}
// join the group
joinGroup := join()
// sync the group
_, err := conn.syncGroup(syncGroupRequestV0{
GroupID: groupID,
GenerationID: joinGroup.GenerationID,
MemberID: joinGroup.MemberID,
GroupAssignments: []syncGroupRequestGroupAssignmentV0{
{
MemberID: joinGroup.MemberID,
MemberAssignments: []byte("blah"),
},
},
})
if err != nil {
t.Fatalf("bad syncGroup: %s", err)
}
generationID = joinGroup.GenerationID
memberID = joinGroup.MemberID
stop = func() {
conn.leaveGroup(leaveGroupRequestV0{
GroupID: groupID,
MemberID: joinGroup.MemberID,
})
}
return
}
func testConnFindCoordinator(t *testing.T, conn *Conn) {
groupID := makeGroupID()
for attempt := 0; attempt < 10; attempt++ {
if attempt != 0 {
time.Sleep(time.Millisecond * 50)
}
response, err := conn.findCoordinator(findCoordinatorRequestV0{CoordinatorKey: groupID})
if err != nil {
if errors.Is(err, GroupCoordinatorNotAvailable) {
continue
}
t.Fatalf("bad findCoordinator: %s", err)
}
if response.Coordinator.NodeID == 0 {
t.Errorf("bad NodeID")
}
if response.Coordinator.Host == "" {
t.Errorf("bad Host")
}
if response.Coordinator.Port == 0 {
t.Errorf("bad Port")
}
return
}
}
func testConnJoinGroupInvalidGroupID(t *testing.T, conn *Conn) {
_, err := conn.joinGroup(joinGroupRequestV1{})
if !errors.Is(err, InvalidGroupId) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", InvalidGroupId, NotCoordinatorForGroup, err)
}
}
func testConnJoinGroupInvalidSessionTimeout(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)
_, err := conn.joinGroup(joinGroupRequestV1{
GroupID: groupID,
})
if !errors.Is(err, InvalidSessionTimeout) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", InvalidSessionTimeout, NotCoordinatorForGroup, err)
}
}
func testConnJoinGroupInvalidRefreshTimeout(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)
_, err := conn.joinGroup(joinGroupRequestV1{
GroupID: groupID,
SessionTimeout: int32(3 * time.Second / time.Millisecond),
})
if !errors.Is(err, InvalidSessionTimeout) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", InvalidSessionTimeout, NotCoordinatorForGroup, err)
}
}
func testConnHeartbeatErr(t *testing.T, conn *Conn) {
groupID := makeGroupID()
createGroup(t, conn, groupID)
_, err := conn.syncGroup(syncGroupRequestV0{
GroupID: groupID,
})
if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
}
}
func testConnLeaveGroupErr(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)
_, err := conn.leaveGroup(leaveGroupRequestV0{
GroupID: groupID,
})
if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
}
}
func testConnSyncGroupErr(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)
_, err := conn.syncGroup(syncGroupRequestV0{
GroupID: groupID,
})
if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
}
}
func testConnListGroupsReturnsGroups(t *testing.T, conn *Conn) {
group1 := makeGroupID()
_, _, stop1 := createGroup(t, conn, group1)
defer stop1()
group2 := makeGroupID()
_, _, stop2 := createGroup(t, conn, group2)
defer stop2()
out, err := conn.listGroups(listGroupsRequestV1{})
if err != nil {
t.Fatalf("bad err: %v", err)
}
containsGroup := func(groupID string) bool {
for _, group := range out.Groups {
if group.GroupID == groupID {
return true
}
}
return false
}
if !containsGroup(group1) {
t.Errorf("expected groups to contain group1")
}
if !containsGroup(group2) {
t.Errorf("expected groups to contain group2")
}
}
func testConnFetchAndCommitOffsets(t *testing.T, conn *Conn) {
const N = 10
if _, err := conn.WriteMessages(makeTestSequence(N)...); err != nil {
t.Fatal(err)
}
groupID := makeGroupID()
generationID, memberID, stop := createGroup(t, conn, groupID)
defer stop()
request := offsetFetchRequestV1{
GroupID: groupID,
Topics: []offsetFetchRequestV1Topic{
{
Topic: conn.topic,
Partitions: []int32{0},
},
},
}
fetch, err := conn.offsetFetch(request)
if err != nil {
t.Fatalf("bad err: %v", err)
}
if v := len(fetch.Responses); v != 1 {
t.Fatalf("expected 1 Response; got %v", v)
}
if v := len(fetch.Responses[0].PartitionResponses); v != 1 {
t.Fatalf("expected 1 PartitionResponses; got %v", v)
}
if offset := fetch.Responses[0].PartitionResponses[0].Offset; offset != -1 {
t.Fatalf("expected initial offset of -1; got %v", offset)
}
committedOffset := int64(N - 1)
_, err = conn.offsetCommit(offsetCommitRequestV2{
GroupID: groupID,
GenerationID: generationID,
MemberID: memberID,
RetentionTime: int64(time.Hour / time.Millisecond),
Topics: []offsetCommitRequestV2Topic{
{
Topic: conn.topic,
Partitions: []offsetCommitRequestV2Partition{
{
Partition: 0,
Offset: committedOffset,
},
},
},
},
})
if err != nil {
t.Fatalf("bad error: %v", err)
}
fetch, err = conn.offsetFetch(request)
if err != nil {
t.Fatalf("bad error: %v", err)
}
fetchedOffset := fetch.Responses[0].PartitionResponses[0].Offset
if committedOffset != fetchedOffset {
t.Fatalf("bad offset. expected %v; got %v", committedOffset, fetchedOffset)
}
}
func testConnWriteReadConcurrently(t *testing.T, conn *Conn) {
const N = 1000
msgs := make([]string, N)
done := make(chan struct{})
written := make(chan struct{}, N/10)
for i := 0; i != N; i++ {
msgs[i] = strconv.Itoa(i)
}
go func() {
defer close(done)
for _, msg := range msgs {
if _, err := conn.Write([]byte(msg)); err != nil {
t.Error(err)
}
written <- struct{}{}
}
}()
b := make([]byte, 128)
for i := 0; i != N; i++ {
// wait until at least one message has been written. the reason for
// this synchronization is that we aren't using deadlines. as such, if
// the read happens before a message is available, it will cause a
// deadlock because the read request will never hit the one byte minimum
// in order to return and release the lock on the conn. by ensuring
// that there's at least one message produced, we don't hit that
// condition.
<-written
n, err := conn.Read(b)
if err != nil {
t.Error(err)
}
if s := string(b[:n]); s != strconv.Itoa(i) {
t.Errorf("bad message read at offset %d: %s", i, s)
}
}
<-done
}
func testConnReadShortBuffer(t *testing.T, conn *Conn) {
if _, err := conn.Write([]byte("Hello World!")); err != nil {
t.Fatal(err)
}
b := make([]byte, 4)
for i := 0; i != 10; i++ {
b[0] = 0
b[1] = 0
b[2] = 0
b[3] = 0
n, err := conn.Read(b)
if !errors.Is(err, io.ErrShortBuffer) {
t.Error("bad error:", i, err)
}