github.com/deanMdreon/kafka-go@v0.4.32/client_test.go (about) 1 package kafka 2 3 import ( 4 "bytes" 5 "context" 6 "io" 7 "math/rand" 8 "net" 9 "testing" 10 "time" 11 12 "github.com/deanMdreon/kafka-go/compress" 13 ktesting "github.com/deanMdreon/kafka-go/testing" 14 ) 15 16 func newLocalClientAndTopic() (*Client, string, func()) { 17 topic := makeTopic() 18 client, shutdown := newLocalClientWithTopic(topic, 1) 19 return client, topic, shutdown 20 } 21 22 func newLocalClientWithTopic(topic string, partitions int) (*Client, func()) { 23 client, shutdown := newLocalClient() 24 if err := clientCreateTopic(client, topic, partitions); err != nil { 25 shutdown() 26 panic(err) 27 } 28 return client, func() { 29 client.DeleteTopics(context.Background(), &DeleteTopicsRequest{ 30 Topics: []string{topic}, 31 }) 32 shutdown() 33 } 34 } 35 36 func clientCreateTopic(client *Client, topic string, partitions int) error { 37 _, err := client.CreateTopics(context.Background(), &CreateTopicsRequest{ 38 Topics: []TopicConfig{{ 39 Topic: topic, 40 NumPartitions: partitions, 41 ReplicationFactor: 1, 42 }}, 43 }) 44 if err != nil { 45 return err 46 } 47 48 // Topic creation seems to be asynchronous. Metadata for the topic partition 49 // layout in the cluster is available in the controller before being synced 50 // with the other brokers, which causes "Error:[3] Unknown Topic Or Partition" 51 // when sending requests to the partition leaders. 52 // 53 // This loop will wait up to 2 seconds polling the cluster until no errors 54 // are returned. 55 for i := 0; i < 20; i++ { 56 r, err := client.Fetch(context.Background(), &FetchRequest{ 57 Topic: topic, 58 Partition: 0, 59 Offset: 0, 60 }) 61 if err == nil && r.Error == nil { 62 break 63 } 64 time.Sleep(100 * time.Millisecond) 65 } 66 67 return nil 68 } 69 70 func clientEndTxn(client *Client, req *EndTxnRequest) error { 71 ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) 72 defer cancel() 73 resp, err := client.EndTxn(ctx, req) 74 if err != nil { 75 return err 76 } 77 78 return resp.Error 79 } 80 81 func newLocalClient() (*Client, func()) { 82 return newClient(TCP("localhost")) 83 } 84 85 func newClient(addr net.Addr) (*Client, func()) { 86 conns := &ktesting.ConnWaitGroup{ 87 DialFunc: (&net.Dialer{}).DialContext, 88 } 89 90 transport := &Transport{ 91 Dial: conns.Dial, 92 Resolver: NewBrokerResolver(nil), 93 } 94 95 client := &Client{ 96 Addr: addr, 97 Timeout: 5 * time.Second, 98 Transport: transport, 99 } 100 101 return client, func() { transport.CloseIdleConnections(); conns.Wait() } 102 } 103 104 func TestClient(t *testing.T) { 105 tests := []struct { 106 scenario string 107 function func(*testing.T, context.Context, *Client) 108 }{ 109 { 110 scenario: "retrieve committed offsets for a consumer group and topic", 111 function: testConsumerGroupFetchOffsets, 112 }, 113 } 114 115 for _, test := range tests { 116 testFunc := test.function 117 t.Run(test.scenario, func(t *testing.T) { 118 ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) 119 defer cancel() 120 121 client, shutdown := newLocalClient() 122 defer shutdown() 123 124 testFunc(t, ctx, client) 125 }) 126 } 127 } 128 129 func testConsumerGroupFetchOffsets(t *testing.T, ctx context.Context, client *Client) { 130 const totalMessages = 144 131 const partitions = 12 132 const msgPerPartition = totalMessages / partitions 133 134 topic := makeTopic() 135 if err := clientCreateTopic(client, topic, partitions); err != nil { 136 t.Fatal(err) 137 } 138 139 groupId := makeGroupID() 140 brokers := []string{"localhost:9092"} 141 142 writer := &Writer{ 143 Addr: TCP(brokers...), 144 Topic: topic, 145 Balancer: &RoundRobin{}, 146 BatchSize: 1, 147 Transport: client.Transport, 148 } 149 if err := writer.WriteMessages(ctx, makeTestSequence(totalMessages)...); err != nil { 150 t.Fatalf("bad write messages: %v", err) 151 } 152 if err := writer.Close(); err != nil { 153 t.Fatalf("bad write err: %v", err) 154 } 155 156 r := NewReader(ReaderConfig{ 157 Brokers: brokers, 158 Topic: topic, 159 GroupID: groupId, 160 MinBytes: 1, 161 MaxBytes: 10e6, 162 MaxWait: 100 * time.Millisecond, 163 }) 164 defer r.Close() 165 166 for i := 0; i < totalMessages; i++ { 167 m, err := r.FetchMessage(ctx) 168 if err != nil { 169 t.Fatalf("error fetching message: %s", err) 170 } 171 if err := r.CommitMessages(context.Background(), m); err != nil { 172 t.Fatal(err) 173 } 174 } 175 176 offsets, err := client.ConsumerOffsets(ctx, TopicAndGroup{GroupId: groupId, Topic: topic}) 177 if err != nil { 178 t.Fatal(err) 179 } 180 181 if len(offsets) != partitions { 182 t.Fatalf("expected %d partitions but only received offsets for %d", partitions, len(offsets)) 183 } 184 185 for i := 0; i < partitions; i++ { 186 committedOffset := offsets[i] 187 if committedOffset != msgPerPartition { 188 t.Errorf("expected partition %d with committed offset of %d but received %d", i, msgPerPartition, committedOffset) 189 } 190 } 191 } 192 193 func TestClientProduceAndConsume(t *testing.T) { 194 ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) 195 defer cancel() 196 // Tests a typical kafka use case, data is produced to a partition, 197 // then consumed back sequentially. We use snappy compression because 198 // kafka stream are often compressed, and verify that each record 199 // produced is exposed to the consumer, and order is preserved. 200 client, topic, shutdown := newLocalClientAndTopic() 201 defer shutdown() 202 203 epoch := time.Now() 204 seed := int64(0) // deterministic 205 prng := rand.New(rand.NewSource(seed)) 206 offset := int64(0) 207 208 const numBatches = 100 209 const recordsPerBatch = 320 210 t.Logf("producing %d batches of %d records...", numBatches, recordsPerBatch) 211 212 for i := 0; i < numBatches; i++ { // produce 100 batches 213 records := make([]Record, recordsPerBatch) 214 215 for i := range records { 216 v := make([]byte, prng.Intn(999)+1) 217 io.ReadFull(prng, v) 218 records[i].Time = epoch 219 records[i].Value = NewBytes(v) 220 } 221 222 res, err := client.Produce(ctx, &ProduceRequest{ 223 Topic: topic, 224 Partition: 0, 225 RequiredAcks: -1, 226 Records: NewRecordReader(records...), 227 Compression: compress.Snappy, 228 }) 229 if err != nil { 230 t.Fatal(err) 231 } 232 if res.Error != nil { 233 t.Fatal(res.Error) 234 } 235 if res.BaseOffset != offset { 236 t.Fatalf("records were produced at an unexpected offset, want %d but got %d", offset, res.BaseOffset) 237 } 238 offset += int64(len(records)) 239 } 240 241 prng.Seed(seed) 242 offset = 0 // reset 243 numFetches := 0 244 numRecords := 0 245 246 for numRecords < (numBatches * recordsPerBatch) { 247 res, err := client.Fetch(ctx, &FetchRequest{ 248 Topic: topic, 249 Partition: 0, 250 Offset: offset, 251 MinBytes: 1, 252 MaxBytes: 256 * 1024, 253 MaxWait: 100 * time.Millisecond, // should only hit on the last fetch 254 }) 255 if err != nil { 256 t.Fatal(err) 257 } 258 if res.Error != nil { 259 t.Fatal(err) 260 } 261 262 for { 263 r, err := res.Records.ReadRecord() 264 if err != nil { 265 if err != io.EOF { 266 t.Fatal(err) 267 } 268 break 269 } 270 271 if r.Key != nil { 272 r.Key.Close() 273 t.Error("unexpected non-null key on record at offset", r.Offset) 274 } 275 276 n := prng.Intn(999) + 1 277 a := make([]byte, n) 278 b := make([]byte, n) 279 io.ReadFull(prng, a) 280 281 _, err = io.ReadFull(r.Value, b) 282 r.Value.Close() 283 if err != nil { 284 t.Fatalf("reading record at offset %d: %v", r.Offset, err) 285 } 286 287 if !bytes.Equal(a, b) { 288 t.Fatalf("value of record at offset %d mismatches", r.Offset) 289 } 290 291 if r.Offset != offset { 292 t.Fatalf("record at offset %d was expected to have offset %d", r.Offset, offset) 293 } 294 295 offset = r.Offset + 1 296 numRecords++ 297 } 298 299 numFetches++ 300 } 301 302 t.Logf("%d records were read in %d fetches", numRecords, numFetches) 303 }