forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
client_test.go
303 lines (259 loc) · 7.14 KB
/
client_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
package kafka
import (
"bytes"
"context"
"io"
"math/rand"
"net"
"testing"
"time"
"github.com/segmentio/kafka-go/compress"
ktesting "github.com/segmentio/kafka-go/testing"
)
func newLocalClientAndTopic() (*Client, string, func()) {
topic := makeTopic()
client, shutdown := newLocalClientWithTopic(topic, 1)
return client, topic, shutdown
}
func newLocalClientWithTopic(topic string, partitions int) (*Client, func()) {
client, shutdown := newLocalClient()
if err := clientCreateTopic(client, topic, partitions); err != nil {
shutdown()
panic(err)
}
return client, func() {
client.DeleteTopics(context.Background(), &DeleteTopicsRequest{
Topics: []string{topic},
})
shutdown()
}
}
func clientCreateTopic(client *Client, topic string, partitions int) error {
_, err := client.CreateTopics(context.Background(), &CreateTopicsRequest{
Topics: []TopicConfig{{
Topic: topic,
NumPartitions: partitions,
ReplicationFactor: 1,
}},
})
if err != nil {
return err
}
// Topic creation seems to be asynchronous. Metadata for the topic partition
// layout in the cluster is available in the controller before being synced
// with the other brokers, which causes "Error:[3] Unknown Topic Or Partition"
// when sending requests to the partition leaders.
//
// This loop will wait up to 2 seconds polling the cluster until no errors
// are returned.
for i := 0; i < 20; i++ {
r, err := client.Fetch(context.Background(), &FetchRequest{
Topic: topic,
Partition: 0,
Offset: 0,
})
if err == nil && r.Error == nil {
break
}
time.Sleep(100 * time.Millisecond)
}
return nil
}
func clientEndTxn(client *Client, req *EndTxnRequest) error {
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()
resp, err := client.EndTxn(ctx, req)
if err != nil {
return err
}
return resp.Error
}
func newLocalClient() (*Client, func()) {
return newClient(TCP("localhost"))
}
func newClient(addr net.Addr) (*Client, func()) {
conns := &ktesting.ConnWaitGroup{
DialFunc: (&net.Dialer{}).DialContext,
}
transport := &Transport{
Dial: conns.Dial,
Resolver: NewBrokerResolver(nil),
}
client := &Client{
Addr: addr,
Timeout: 5 * time.Second,
Transport: transport,
}
return client, func() { transport.CloseIdleConnections(); conns.Wait() }
}
func TestClient(t *testing.T) {
tests := []struct {
scenario string
function func(*testing.T, context.Context, *Client)
}{
{
scenario: "retrieve committed offsets for a consumer group and topic",
function: testConsumerGroupFetchOffsets,
},
}
for _, test := range tests {
testFunc := test.function
t.Run(test.scenario, func(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
client, shutdown := newLocalClient()
defer shutdown()
testFunc(t, ctx, client)
})
}
}
func testConsumerGroupFetchOffsets(t *testing.T, ctx context.Context, client *Client) {
const totalMessages = 144
const partitions = 12
const msgPerPartition = totalMessages / partitions
topic := makeTopic()
if err := clientCreateTopic(client, topic, partitions); err != nil {
t.Fatal(err)
}
groupId := makeGroupID()
brokers := []string{"localhost:9092"}
writer := &Writer{
Addr: TCP(brokers...),
Topic: topic,
Balancer: &RoundRobin{},
BatchSize: 1,
Transport: client.Transport,
}
if err := writer.WriteMessages(ctx, makeTestSequence(totalMessages)...); err != nil {
t.Fatalf("bad write messages: %v", err)
}
if err := writer.Close(); err != nil {
t.Fatalf("bad write err: %v", err)
}
r := NewReader(ReaderConfig{
Brokers: brokers,
Topic: topic,
GroupID: groupId,
MinBytes: 1,
MaxBytes: 10e6,
MaxWait: 100 * time.Millisecond,
})
defer r.Close()
for i := 0; i < totalMessages; i++ {
m, err := r.FetchMessage(ctx)
if err != nil {
t.Fatalf("error fetching message: %s", err)
}
if err := r.CommitMessages(context.Background(), m); err != nil {
t.Fatal(err)
}
}
offsets, err := client.ConsumerOffsets(ctx, TopicAndGroup{GroupId: groupId, Topic: topic})
if err != nil {
t.Fatal(err)
}
if len(offsets) != partitions {
t.Fatalf("expected %d partitions but only received offsets for %d", partitions, len(offsets))
}
for i := 0; i < partitions; i++ {
committedOffset := offsets[i]
if committedOffset != msgPerPartition {
t.Errorf("expected partition %d with committed offset of %d but received %d", i, msgPerPartition, committedOffset)
}
}
}
func TestClientProduceAndConsume(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
// Tests a typical kafka use case, data is produced to a partition,
// then consumed back sequentially. We use snappy compression because
// kafka stream are often compressed, and verify that each record
// produced is exposed to the consumer, and order is preserved.
client, topic, shutdown := newLocalClientAndTopic()
defer shutdown()
epoch := time.Now()
seed := int64(0) // deterministic
prng := rand.New(rand.NewSource(seed))
offset := int64(0)
const numBatches = 100
const recordsPerBatch = 320
t.Logf("producing %d batches of %d records...", numBatches, recordsPerBatch)
for i := 0; i < numBatches; i++ { // produce 100 batches
records := make([]Record, recordsPerBatch)
for i := range records {
v := make([]byte, prng.Intn(999)+1)
io.ReadFull(prng, v)
records[i].Time = epoch
records[i].Value = NewBytes(v)
}
res, err := client.Produce(ctx, &ProduceRequest{
Topic: topic,
Partition: 0,
RequiredAcks: -1,
Records: NewRecordReader(records...),
Compression: compress.Snappy,
})
if err != nil {
t.Fatal(err)
}
if res.Error != nil {
t.Fatal(res.Error)
}
if res.BaseOffset != offset {
t.Fatalf("records were produced at an unexpected offset, want %d but got %d", offset, res.BaseOffset)
}
offset += int64(len(records))
}
prng.Seed(seed)
offset = 0 // reset
numFetches := 0
numRecords := 0
for numRecords < (numBatches * recordsPerBatch) {
res, err := client.Fetch(ctx, &FetchRequest{
Topic: topic,
Partition: 0,
Offset: offset,
MinBytes: 1,
MaxBytes: 256 * 1024,
MaxWait: 100 * time.Millisecond, // should only hit on the last fetch
})
if err != nil {
t.Fatal(err)
}
if res.Error != nil {
t.Fatal(err)
}
for {
r, err := res.Records.ReadRecord()
if err != nil {
if err != io.EOF {
t.Fatal(err)
}
break
}
if r.Key != nil {
r.Key.Close()
t.Error("unexpected non-null key on record at offset", r.Offset)
}
n := prng.Intn(999) + 1
a := make([]byte, n)
b := make([]byte, n)
io.ReadFull(prng, a)
_, err = io.ReadFull(r.Value, b)
r.Value.Close()
if err != nil {
t.Fatalf("reading record at offset %d: %v", r.Offset, err)
}
if !bytes.Equal(a, b) {
t.Fatalf("value of record at offset %d mismatches", r.Offset)
}
if r.Offset != offset {
t.Fatalf("record at offset %d was expected to have offset %d", r.Offset, offset)
}
offset = r.Offset + 1
numRecords++
}
numFetches++
}
t.Logf("%d records were read in %d fetches", numRecords, numFetches)
}