github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/integration_test.go (about) 1 /** 2 * Copyright 2016 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package kafka 18 19 import ( 20 "context" 21 "encoding/binary" 22 "fmt" 23 "math/rand" 24 "path" 25 "reflect" 26 "runtime" 27 "sort" 28 "testing" 29 "time" 30 ) 31 32 // producer test control 33 type producerCtrl struct { 34 silent bool 35 withDr bool // use delivery channel 36 batchProducer bool // enable batch producer 37 } 38 39 // define commitMode with constants 40 type commitMode string 41 42 const ( 43 ViaCommitMessageAPI = "CommitMessage" 44 ViaCommitOffsetsAPI = "CommitOffsets" 45 ViaCommitAPI = "Commit" 46 ) 47 48 // consumer test control 49 type consumerCtrl struct { 50 autoCommit bool // set enable.auto.commit property 51 useChannel bool 52 commitMode commitMode // which commit api to use 53 } 54 55 type testmsgType struct { 56 msg Message 57 expectedError Error 58 } 59 60 // msgtracker tracks messages 61 type msgtracker struct { 62 t *testing.T 63 msgcnt int64 64 errcnt int64 // count of failed messages 65 msgs []*Message 66 } 67 68 // msgtrackerStart sets up a new message tracker 69 func msgtrackerStart(t *testing.T, expectedCnt int) (mt msgtracker) { 70 mt = msgtracker{t: t} 71 mt.msgs = make([]*Message, expectedCnt) 72 return mt 73 } 74 75 var testMsgsInit = false 76 var p0TestMsgs []*testmsgType // partition 0 test messages 77 // pAllTestMsgs holds messages for various partitions including PartitionAny and invalid partitions 78 var pAllTestMsgs []*testmsgType 79 80 // createTestMessages populates p0TestMsgs and pAllTestMsgs 81 func createTestMessages() { 82 83 if testMsgsInit { 84 return 85 } 86 defer func() { testMsgsInit = true }() 87 88 testmsgs := make([]*testmsgType, 100) 89 i := 0 90 91 // a test message with default initialization 92 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}}} 93 i++ 94 95 // a test message for partition 0 with only Opaque specified 96 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 97 Opaque: fmt.Sprintf("Op%d", i), 98 }} 99 i++ 100 101 // a test message for partition 0 with empty Value and Keys 102 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 103 Value: []byte(""), 104 Key: []byte(""), 105 Opaque: fmt.Sprintf("Op%d", i), 106 }} 107 i++ 108 109 // a test message for partition 0 with Value, Key, and Opaque 110 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 111 Value: []byte(fmt.Sprintf("value%d", i)), 112 Key: []byte(fmt.Sprintf("key%d", i)), 113 Opaque: fmt.Sprintf("Op%d", i), 114 }} 115 i++ 116 117 // a test message for partition 0 without Value 118 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 119 Key: []byte(fmt.Sprintf("key%d", i)), 120 Opaque: fmt.Sprintf("Op%d", i), 121 }} 122 i++ 123 124 // a test message for partition 0 without Key 125 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 126 Value: []byte(fmt.Sprintf("value%d", i)), 127 Opaque: fmt.Sprintf("Op%d", i), 128 }} 129 i++ 130 131 p0TestMsgs = testmsgs[:i] 132 133 // a test message for PartitonAny with Value, Key, and Opaque 134 testmsgs[i] = &testmsgType{msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: PartitionAny}, 135 Value: []byte(fmt.Sprintf("value%d", i)), 136 Key: []byte(fmt.Sprintf("key%d", i)), 137 Opaque: fmt.Sprintf("Op%d", i), 138 }} 139 i++ 140 141 // a test message for a non-existent partition with Value, Key, and Opaque. 142 // It should generate ErrUnknownPartition 143 testmsgs[i] = &testmsgType{expectedError: Error{code: ErrUnknownPartition}, 144 msg: Message{TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: int32(10000)}, 145 Value: []byte(fmt.Sprintf("value%d", i)), 146 Key: []byte(fmt.Sprintf("key%d", i)), 147 Opaque: fmt.Sprintf("Op%d", i), 148 }} 149 i++ 150 151 pAllTestMsgs = testmsgs[:i] 152 } 153 154 // consume messages through the Poll() interface 155 func eventTestPollConsumer(c *Consumer, mt *msgtracker, expCnt int) { 156 for true { 157 ev := c.Poll(100) 158 if ev == nil { 159 // timeout 160 continue 161 } 162 if !handleTestEvent(c, mt, expCnt, ev) { 163 break 164 } 165 } 166 } 167 168 // consume messages through the Events channel 169 func eventTestChannelConsumer(c *Consumer, mt *msgtracker, expCnt int) { 170 for ev := range c.Events() { 171 if !handleTestEvent(c, mt, expCnt, ev) { 172 break 173 } 174 } 175 } 176 177 // handleTestEvent returns false if processing should stop, else true. Tracks the message received 178 func handleTestEvent(c *Consumer, mt *msgtracker, expCnt int, ev Event) bool { 179 switch e := ev.(type) { 180 case *Message: 181 if e.TopicPartition.Error != nil { 182 mt.t.Errorf("Error: %v", e.TopicPartition) 183 } 184 mt.msgs[mt.msgcnt] = e 185 mt.msgcnt++ 186 if mt.msgcnt >= int64(expCnt) { 187 return false 188 } 189 case PartitionEOF: 190 break // silence 191 default: 192 mt.t.Fatalf("Consumer error: %v", e) 193 } 194 return true 195 196 } 197 198 // delivery event handler. Tracks the message received 199 func deliveryTestHandler(t *testing.T, expCnt int64, deliveryChan chan Event, mt *msgtracker, doneChan chan int64) { 200 201 for ev := range deliveryChan { 202 m, ok := ev.(*Message) 203 if !ok { 204 continue 205 } 206 207 mt.msgs[mt.msgcnt] = m 208 mt.msgcnt++ 209 210 if m.TopicPartition.Error != nil { 211 mt.errcnt++ 212 // log it and check it later 213 t.Logf("Message delivery error: %v", m.TopicPartition) 214 } 215 216 t.Logf("Delivered %d/%d to %s, error count %d", mt.msgcnt, expCnt, m.TopicPartition, mt.errcnt) 217 218 if mt.msgcnt >= expCnt { 219 break 220 } 221 222 } 223 224 doneChan <- mt.msgcnt 225 close(doneChan) 226 } 227 228 // producerTest produces messages in <testmsgs> to topic. Verifies delivered messages 229 func producerTest(t *testing.T, testname string, testmsgs []*testmsgType, pc producerCtrl, produceFunc func(p *Producer, m *Message, drChan chan Event)) { 230 231 if !testconfRead() { 232 t.Skipf("Missing testconf.json") 233 } 234 235 if testmsgs == nil { 236 createTestMessages() 237 testmsgs = pAllTestMsgs 238 } 239 240 //get the number of messages prior to producing more messages 241 prerunMsgCnt, err := getMessageCountInTopic(testconf.Topic) 242 if err != nil { 243 t.Fatalf("Cannot get message count, Error: %s\n", err) 244 } 245 246 conf := ConfigMap{"bootstrap.servers": testconf.Brokers, 247 "go.batch.producer": pc.batchProducer, 248 "go.delivery.reports": pc.withDr, 249 "queue.buffering.max.messages": len(testmsgs), 250 "acks": "all"} 251 252 conf.updateFromTestconf() 253 254 p, err := NewProducer(&conf) 255 if err != nil { 256 panic(err) 257 } 258 259 mt := msgtrackerStart(t, len(testmsgs)) 260 261 var doneChan chan int64 262 var drChan chan Event 263 264 if pc.withDr { 265 doneChan = make(chan int64) 266 drChan = p.Events() 267 go deliveryTestHandler(t, int64(len(testmsgs)), p.Events(), &mt, doneChan) 268 } 269 270 if !pc.silent { 271 t.Logf("%s: produce %d messages", testname, len(testmsgs)) 272 } 273 274 for i := 0; i < len(testmsgs); i++ { 275 t.Logf("producing message %d: %v\n", i, testmsgs[i].msg) 276 produceFunc(p, &testmsgs[i].msg, drChan) 277 } 278 279 if !pc.silent { 280 t.Logf("produce done") 281 } 282 283 // Wait for messages in-flight and in-queue to get delivered. 284 if !pc.silent { 285 t.Logf("%s: %d messages in queue", testname, p.Len()) 286 } 287 288 r := p.Flush(10000) 289 if r > 0 { 290 t.Errorf("%s: %d messages remains in queue after Flush()", testname, r) 291 } 292 293 if pc.withDr { 294 mt.msgcnt = <-doneChan 295 } else { 296 mt.msgcnt = int64(len(testmsgs)) 297 } 298 299 if !pc.silent { 300 t.Logf("delivered %d messages\n", mt.msgcnt) 301 } 302 303 p.Close() 304 305 //get the number of messages afterward 306 postrunMsgCnt, err := getMessageCountInTopic(testconf.Topic) 307 if err != nil { 308 t.Fatalf("Cannot get message count, Error: %s\n", err) 309 } 310 311 if !pc.silent { 312 t.Logf("prerun message count: %d, postrun count %d, delta: %d\n", prerunMsgCnt, postrunMsgCnt, postrunMsgCnt-prerunMsgCnt) 313 t.Logf("deliveried message count: %d, error message count %d\n", mt.msgcnt, mt.errcnt) 314 315 } 316 317 // verify the count and messages only if we get the delivered messages 318 if pc.withDr { 319 if int64(postrunMsgCnt-prerunMsgCnt) != (mt.msgcnt - mt.errcnt) { 320 t.Errorf("Expected topic message count %d, got %d\n", prerunMsgCnt+int(mt.msgcnt-mt.errcnt), postrunMsgCnt) 321 } 322 323 verifyMessages(t, mt.msgs, testmsgs) 324 } 325 } 326 327 // consumerTest consumes messages from a pre-primed (produced to) topic. 328 // assignmentStrategy may be "" to use the default strategy. 329 func consumerTest(t *testing.T, testname string, assignmentStrategy string, msgcnt int, cc consumerCtrl, consumeFunc func(c *Consumer, mt *msgtracker, expCnt int), rebalanceCb func(c *Consumer, event Event) error) { 330 331 if msgcnt == 0 { 332 createTestMessages() 333 producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{}, 334 func(p *Producer, m *Message, drChan chan Event) { 335 p.ProduceChannel() <- m 336 }) 337 msgcnt = len(p0TestMsgs) 338 } 339 340 conf := ConfigMap{"bootstrap.servers": testconf.Brokers, 341 "go.events.channel.enable": cc.useChannel, 342 "group.id": testconf.GroupID + 343 fmt.Sprintf("-%d", rand.Intn(1000000)), 344 "session.timeout.ms": 6000, 345 "api.version.request": "true", 346 "enable.auto.commit": cc.autoCommit, 347 "debug": ",", 348 "auto.offset.reset": "earliest"} 349 if assignmentStrategy != "" { 350 conf["partition.assignment.strategy"] = assignmentStrategy 351 } 352 353 conf.updateFromTestconf() 354 355 c, err := NewConsumer(&conf) 356 357 if err != nil { 358 panic(err) 359 } 360 defer c.Close() 361 362 expCnt := msgcnt 363 mt := msgtrackerStart(t, expCnt) 364 365 t.Logf("%s, expecting %d messages", testname, expCnt) 366 c.Subscribe(testconf.Topic, rebalanceCb) 367 368 consumeFunc(c, &mt, expCnt) 369 370 //test commits 371 switch cc.commitMode { 372 case ViaCommitMessageAPI: 373 // verify CommitMessage() API 374 for _, message := range mt.msgs { 375 _, commitErr := c.CommitMessage(message) 376 if commitErr != nil { 377 t.Errorf("Cannot commit message. Error: %s\n", commitErr) 378 } 379 } 380 case ViaCommitOffsetsAPI: 381 // verify CommitOffset 382 partitions := make([]TopicPartition, len(mt.msgs)) 383 for index, message := range mt.msgs { 384 partitions[index] = message.TopicPartition 385 } 386 _, commitErr := c.CommitOffsets(partitions) 387 if commitErr != nil { 388 t.Errorf("Failed to commit using CommitOffsets. Error: %s\n", commitErr) 389 } 390 case ViaCommitAPI: 391 // verify Commit() API 392 _, commitErr := c.Commit() 393 if commitErr != nil { 394 t.Errorf("Failed to commit. Error: %s", commitErr) 395 } 396 397 } 398 399 // Trigger RevokePartitions 400 c.Unsubscribe() 401 402 // Handle RevokePartitions 403 c.Poll(500) 404 405 } 406 407 //Test consumer QueryWatermarkOffsets API 408 func TestConsumerQueryWatermarkOffsets(t *testing.T) { 409 if !testconfRead() { 410 t.Skipf("Missing testconf.json") 411 } 412 413 // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to 414 // get the number of messages in a topic 415 msgcnt, err := getMessageCountInTopic(testconf.Topic) 416 if err != nil { 417 t.Errorf("Cannot get message size. Error: %s\n", err) 418 } 419 420 // Prime topic with test messages 421 createTestMessages() 422 producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, 423 func(p *Producer, m *Message, drChan chan Event) { 424 p.ProduceChannel() <- m 425 }) 426 427 // getMessageCountInTopic() uses consumer QueryWatermarkOffsets() API to 428 // get the number of messages in a topic 429 newmsgcnt, err := getMessageCountInTopic(testconf.Topic) 430 if err != nil { 431 t.Errorf("Cannot get message size. Error: %s\n", err) 432 } 433 434 if newmsgcnt-msgcnt != len(p0TestMsgs) { 435 t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newmsgcnt-msgcnt) 436 } 437 438 } 439 440 //Test consumer GetWatermarkOffsets API 441 func TestConsumerGetWatermarkOffsets(t *testing.T) { 442 if !testconfRead() { 443 t.Skipf("Missing testconf.json") 444 } 445 446 // Create consumer 447 config := &ConfigMap{ 448 "go.events.channel.enable": true, 449 "bootstrap.servers": testconf.Brokers, 450 "group.id": testconf.GroupID, 451 "session.timeout.ms": 6000, 452 "enable.auto.commit": false, 453 "auto.offset.reset": "earliest", 454 } 455 _ = config.updateFromTestconf() 456 457 c, err := NewConsumer(config) 458 if err != nil { 459 t.Fatalf("Unable to create consumer: %s", err) 460 } 461 defer func() { _ = c.Close() }() 462 463 err = c.Subscribe(testconf.Topic, nil) 464 465 // Prime topic with test messages 466 createTestMessages() 467 producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, 468 func(p *Producer, m *Message, drChan chan Event) { 469 p.ProduceChannel() <- m 470 }) 471 472 // Wait for messages to be received so that we know the watermark offsets have been delivered 473 // with the fetch response 474 for ev := range c.Events() { 475 if _, ok := ev.(*Message); ok { 476 break 477 } 478 } 479 480 _, queryHigh, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) 481 if err != nil { 482 t.Fatalf("Error querying watermark offsets: %s", err) 483 } 484 485 // We are not currently testing the low watermark offset as it only gets set every 10s by the stats timer 486 _, getHigh, err := c.GetWatermarkOffsets(testconf.Topic, 0) 487 if err != nil { 488 t.Fatalf("Error getting watermark offsets: %s", err) 489 } 490 491 if queryHigh != getHigh { 492 t.Errorf("QueryWatermarkOffsets high[%d] does not equal GetWatermarkOffsets high[%d]", queryHigh, getHigh) 493 } 494 495 } 496 497 //TestConsumerOffsetsForTimes 498 func TestConsumerOffsetsForTimes(t *testing.T) { 499 if !testconfRead() { 500 t.Skipf("Missing testconf.json") 501 } 502 503 conf := ConfigMap{"bootstrap.servers": testconf.Brokers, 504 "group.id": testconf.GroupID, 505 "api.version.request": true} 506 507 conf.updateFromTestconf() 508 509 c, err := NewConsumer(&conf) 510 511 if err != nil { 512 panic(err) 513 } 514 defer c.Close() 515 516 // Prime topic with test messages 517 createTestMessages() 518 producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, 519 func(p *Producer, m *Message, drChan chan Event) { 520 p.ProduceChannel() <- m 521 }) 522 523 times := make([]TopicPartition, 1) 524 times[0] = TopicPartition{Topic: &testconf.Topic, Partition: 0, Offset: 12345} 525 offsets, err := c.OffsetsForTimes(times, 5000) 526 if err != nil { 527 t.Errorf("OffsetsForTimes() failed: %s\n", err) 528 return 529 } 530 531 if len(offsets) != 1 { 532 t.Errorf("OffsetsForTimes() returned wrong length %d, expected 1\n", len(offsets)) 533 return 534 } 535 536 if *offsets[0].Topic != testconf.Topic || offsets[0].Partition != 0 { 537 t.Errorf("OffsetsForTimes() returned wrong topic/partition\n") 538 return 539 } 540 541 if offsets[0].Error != nil { 542 t.Errorf("OffsetsForTimes() returned error for partition 0: %s\n", err) 543 return 544 } 545 546 low, _, err := c.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) 547 if err != nil { 548 t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) 549 return 550 } 551 552 t.Logf("OffsetsForTimes() returned offset %d for timestamp %d\n", offsets[0].Offset, times[0].Offset) 553 554 // Since we're using a phony low timestamp it is assumed that the returned 555 // offset will be oldest message. 556 if offsets[0].Offset != Offset(low) { 557 t.Errorf("OffsetsForTimes() returned invalid offset %d for timestamp %d, expected %d\n", offsets[0].Offset, times[0].Offset, low) 558 return 559 } 560 561 } 562 563 // test consumer GetMetadata API 564 func TestConsumerGetMetadata(t *testing.T) { 565 if !testconfRead() { 566 t.Skipf("Missing testconf.json") 567 } 568 569 config := &ConfigMap{"bootstrap.servers": testconf.Brokers, 570 "group.id": testconf.GroupID} 571 config.updateFromTestconf() 572 573 // Create consumer 574 c, err := NewConsumer(config) 575 if err != nil { 576 t.Errorf("Failed to create consumer: %s\n", err) 577 return 578 } 579 defer c.Close() 580 581 metaData, err := c.GetMetadata(&testconf.Topic, false, 5*1000) 582 if err != nil { 583 t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) 584 return 585 } 586 t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) 587 588 metaData, err = c.GetMetadata(nil, true, 5*1000) 589 if err != nil { 590 t.Errorf("Failed to get meta data, Error: %s\n", err) 591 return 592 } 593 t.Logf("Meta data for consumer: %v\n", metaData) 594 } 595 596 //Test producer QueryWatermarkOffsets API 597 func TestProducerQueryWatermarkOffsets(t *testing.T) { 598 if !testconfRead() { 599 t.Skipf("Missing testconf.json") 600 } 601 602 config := &ConfigMap{"bootstrap.servers": testconf.Brokers} 603 config.updateFromTestconf() 604 605 // Create producer 606 p, err := NewProducer(config) 607 if err != nil { 608 t.Errorf("Failed to create producer: %s\n", err) 609 return 610 } 611 defer p.Close() 612 613 low, high, err := p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) 614 if err != nil { 615 t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) 616 return 617 } 618 cnt := high - low 619 t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) 620 621 createTestMessages() 622 producerTest(t, "Priming producer", p0TestMsgs, producerCtrl{silent: true}, 623 func(p *Producer, m *Message, drChan chan Event) { 624 p.ProduceChannel() <- m 625 }) 626 627 low, high, err = p.QueryWatermarkOffsets(testconf.Topic, 0, 5*1000) 628 if err != nil { 629 t.Errorf("Failed to query watermark offsets for topic %s. Error: %s\n", testconf.Topic, err) 630 return 631 } 632 t.Logf("Watermark offsets fo topic %s: low=%d, high=%d\n", testconf.Topic, low, high) 633 newcnt := high - low 634 t.Logf("count = %d, New count = %d\n", cnt, newcnt) 635 if newcnt-cnt != int64(len(p0TestMsgs)) { 636 t.Errorf("Incorrect offsets. Expected message count %d, got %d\n", len(p0TestMsgs), newcnt-cnt) 637 } 638 } 639 640 //Test producer GetMetadata API 641 func TestProducerGetMetadata(t *testing.T) { 642 if !testconfRead() { 643 t.Skipf("Missing testconf.json") 644 } 645 646 config := &ConfigMap{"bootstrap.servers": testconf.Brokers} 647 config.updateFromTestconf() 648 649 // Create producer 650 p, err := NewProducer(config) 651 if err != nil { 652 t.Errorf("Failed to create producer: %s\n", err) 653 return 654 } 655 defer p.Close() 656 657 metaData, err := p.GetMetadata(&testconf.Topic, false, 5*1000) 658 if err != nil { 659 t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) 660 return 661 } 662 t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) 663 664 metaData, err = p.GetMetadata(nil, true, 5*1000) 665 if err != nil { 666 t.Errorf("Failed to get meta data, Error: %s\n", err) 667 return 668 } 669 t.Logf("Meta data for producer: %v\n", metaData) 670 671 } 672 673 // test producer function-based API without delivery report 674 func TestProducerFunc(t *testing.T) { 675 producerTest(t, "Function producer (without DR)", 676 nil, producerCtrl{}, 677 func(p *Producer, m *Message, drChan chan Event) { 678 err := p.Produce(m, drChan) 679 if err != nil { 680 t.Errorf("Produce() failed: %v", err) 681 } 682 }) 683 } 684 685 // test producer function-based API with delivery report 686 func TestProducerFuncDR(t *testing.T) { 687 producerTest(t, "Function producer (with DR)", 688 nil, producerCtrl{withDr: true}, 689 func(p *Producer, m *Message, drChan chan Event) { 690 err := p.Produce(m, drChan) 691 if err != nil { 692 t.Errorf("Produce() failed: %v", err) 693 } 694 }) 695 } 696 697 // test producer with bad messages 698 func TestProducerWithBadMessages(t *testing.T) { 699 conf := ConfigMap{"bootstrap.servers": testconf.Brokers} 700 conf.updateFromTestconf() 701 702 p, err := NewProducer(&conf) 703 if err != nil { 704 panic(err) 705 } 706 defer p.Close() 707 708 // producing a nil message should return an error without crash 709 err = p.Produce(nil, p.Events()) 710 if err == nil { 711 t.Errorf("Producing a nil message should return error\n") 712 } else { 713 t.Logf("Producing a nil message returns expected error: %s\n", err) 714 } 715 716 // producing a blank message (with nil Topic) should return an error without crash 717 err = p.Produce(&Message{}, p.Events()) 718 if err == nil { 719 t.Errorf("Producing a blank message should return error\n") 720 } else { 721 t.Logf("Producing a blank message returns expected error: %s\n", err) 722 } 723 } 724 725 // test producer channel-based API without delivery report 726 func TestProducerChannel(t *testing.T) { 727 producerTest(t, "Channel producer (without DR)", 728 nil, producerCtrl{}, 729 func(p *Producer, m *Message, drChan chan Event) { 730 p.ProduceChannel() <- m 731 }) 732 } 733 734 // test producer channel-based API with delivery report 735 func TestProducerChannelDR(t *testing.T) { 736 producerTest(t, "Channel producer (with DR)", 737 nil, producerCtrl{withDr: true}, 738 func(p *Producer, m *Message, drChan chan Event) { 739 p.ProduceChannel() <- m 740 }) 741 742 } 743 744 // test batch producer channel-based API without delivery report 745 func TestProducerBatchChannel(t *testing.T) { 746 producerTest(t, "Channel producer (without DR, batch channel)", 747 nil, producerCtrl{batchProducer: true}, 748 func(p *Producer, m *Message, drChan chan Event) { 749 p.ProduceChannel() <- m 750 }) 751 } 752 753 // test batch producer channel-based API with delivery report 754 func TestProducerBatchChannelDR(t *testing.T) { 755 producerTest(t, "Channel producer (DR, batch channel)", 756 nil, producerCtrl{withDr: true, batchProducer: true}, 757 func(p *Producer, m *Message, drChan chan Event) { 758 p.ProduceChannel() <- m 759 }) 760 } 761 762 // use opaque string to locate the matching test message for message verification 763 func findExpectedMessage(expected []*testmsgType, opaque string) *testmsgType { 764 for i, m := range expected { 765 if expected[i].msg.Opaque != nil && expected[i].msg.Opaque.(string) == opaque { 766 return m 767 } 768 } 769 return nil 770 } 771 772 // verify the message content against the expected 773 func verifyMessages(t *testing.T, msgs []*Message, expected []*testmsgType) { 774 if len(msgs) != len(expected) { 775 t.Errorf("Expected %d messages, got %d instead\n", len(expected), len(msgs)) 776 return 777 } 778 for _, m := range msgs { 779 if m.Opaque == nil { 780 continue // No way to look up the corresponding expected message, let it go 781 } 782 testmsg := findExpectedMessage(expected, m.Opaque.(string)) 783 if testmsg == nil { 784 t.Errorf("Cannot find a matching expected message for message %v\n", m) 785 continue 786 } 787 em := testmsg.msg 788 if m.TopicPartition.Error != nil { 789 if m.TopicPartition.Error != testmsg.expectedError { 790 t.Errorf("Expected error %s, but got error %s\n", testmsg.expectedError, m.TopicPartition.Error) 791 } 792 continue 793 } 794 795 // check partition 796 if em.TopicPartition.Partition == PartitionAny { 797 if m.TopicPartition.Partition < 0 { 798 t.Errorf("Expected partition %d, got %d\n", em.TopicPartition.Partition, m.TopicPartition.Partition) 799 } 800 } else if em.TopicPartition.Partition != m.TopicPartition.Partition { 801 t.Errorf("Expected partition %d, got %d\n", em.TopicPartition.Partition, m.TopicPartition.Partition) 802 } 803 804 //check Key, Value, and Opaque 805 if string(m.Key) != string(em.Key) { 806 t.Errorf("Expected Key %v, got %v\n", m.Key, em.Key) 807 } 808 if string(m.Value) != string(em.Value) { 809 t.Errorf("Expected Value %v, got %v\n", m.Value, em.Value) 810 } 811 if m.Opaque.(string) != em.Opaque.(string) { 812 t.Errorf("Expected Opaque %v, got %v\n", m.Opaque, em.Opaque) 813 } 814 815 } 816 } 817 818 // test consumer APIs with various message commit modes 819 func consumerTestWithCommits(t *testing.T, testname string, assignmentStrategy string, msgcnt int, useChannel bool, consumeFunc func(c *Consumer, mt *msgtracker, expCnt int), rebalanceCb func(c *Consumer, event Event) error) { 820 consumerTest(t, testname+" auto commit", assignmentStrategy, 821 msgcnt, consumerCtrl{useChannel: useChannel, autoCommit: true}, consumeFunc, rebalanceCb) 822 823 consumerTest(t, testname+" using CommitMessage() API", assignmentStrategy, 824 msgcnt, consumerCtrl{useChannel: useChannel, commitMode: ViaCommitMessageAPI}, consumeFunc, rebalanceCb) 825 826 consumerTest(t, testname+" using CommitOffsets() API", assignmentStrategy, 827 msgcnt, consumerCtrl{useChannel: useChannel, commitMode: ViaCommitOffsetsAPI}, consumeFunc, rebalanceCb) 828 829 consumerTest(t, testname+" using Commit() API", assignmentStrategy, 830 831 msgcnt, consumerCtrl{useChannel: useChannel, commitMode: ViaCommitAPI}, consumeFunc, rebalanceCb) 832 833 } 834 835 // test consumer channel-based API 836 func TestConsumerChannel(t *testing.T) { 837 consumerTestWithCommits(t, "Channel Consumer", 838 "", 0, true, eventTestChannelConsumer, nil) 839 } 840 841 // test consumer channel-based API with incremental rebalancing 842 func TestConsumerChannelIncremental(t *testing.T) { 843 consumerTestWithCommits(t, "Channel Consumer Incremental", 844 "cooperative-sticky", 0, true, eventTestChannelConsumer, nil) 845 } 846 847 // test consumer poll-based API 848 func TestConsumerPoll(t *testing.T) { 849 consumerTestWithCommits(t, "Poll Consumer", "", 0, false, eventTestPollConsumer, nil) 850 } 851 852 // test consumer poll-based API with incremental rebalancing 853 func TestConsumerPollIncremental(t *testing.T) { 854 consumerTestWithCommits(t, "Poll Consumer ncremental", 855 "cooperative-sticky", 0, false, eventTestPollConsumer, nil) 856 } 857 858 // test consumer poll-based API with rebalance callback 859 func TestConsumerPollRebalance(t *testing.T) { 860 consumerTestWithCommits(t, "Poll Consumer (rebalance callback)", 861 "", 0, false, eventTestPollConsumer, 862 func(c *Consumer, event Event) error { 863 t.Logf("Rebalanced: %s", event) 864 return nil 865 }) 866 } 867 868 // test consumer poll-based API with incremental no-op rebalance callback 869 func TestConsumerPollRebalanceIncrementalNoop(t *testing.T) { 870 consumerTestWithCommits(t, "Poll Consumer (incremental no-op rebalance callback)", 871 "cooperative-sticky", 0, false, eventTestPollConsumer, 872 func(c *Consumer, event Event) error { 873 t.Logf("Rebalanced: %s", event) 874 return nil 875 }) 876 } 877 878 // test consumer poll-based API with incremental rebalance callback 879 func TestConsumerPollRebalanceIncremental(t *testing.T) { 880 consumerTestWithCommits(t, "Poll Consumer (incremental rebalance callback)", 881 "cooperative-sticky", 0, false, eventTestPollConsumer, 882 func(c *Consumer, event Event) error { 883 t.Logf("Rebalanced: %s (RebalanceProtocol=%s, AssignmentLost=%v)", 884 event, c.GetRebalanceProtocol(), c.AssignmentLost()) 885 886 switch e := event.(type) { 887 case AssignedPartitions: 888 err := c.IncrementalAssign(e.Partitions) 889 if err != nil { 890 t.Errorf("IncrementalAssign() failed: %s\n", err) 891 return err 892 } 893 case RevokedPartitions: 894 err := c.IncrementalUnassign(e.Partitions) 895 if err != nil { 896 t.Errorf("IncrementalUnassign() failed: %s\n", err) 897 return err 898 } 899 default: 900 t.Fatalf("Unexpected rebalance event: %v\n", e) 901 } 902 903 return nil 904 }) 905 } 906 907 // Test Committed() API 908 func TestConsumerCommitted(t *testing.T) { 909 consumerTestWithCommits(t, "Poll Consumer (rebalance callback, verify Committed())", 910 "", 0, false, eventTestPollConsumer, 911 func(c *Consumer, event Event) error { 912 t.Logf("Rebalanced: %s", event) 913 rp, ok := event.(RevokedPartitions) 914 if ok { 915 offsets, err := c.Committed(rp.Partitions, 5000) 916 if err != nil { 917 t.Errorf("Failed to get committed offsets: %s\n", err) 918 return nil 919 } 920 921 t.Logf("Retrieved Committed offsets: %s\n", offsets) 922 923 if len(offsets) != len(rp.Partitions) || len(rp.Partitions) == 0 { 924 t.Errorf("Invalid number of partitions %d, should be %d (and >0)\n", len(offsets), len(rp.Partitions)) 925 } 926 927 // Verify proper offsets: at least one partition needs 928 // to have a committed offset. 929 validCnt := 0 930 for _, p := range offsets { 931 if p.Error != nil { 932 t.Errorf("Committed() partition error: %v: %v", p, p.Error) 933 } else if p.Offset >= 0 { 934 validCnt++ 935 } 936 } 937 938 if validCnt == 0 { 939 t.Errorf("Committed(): no partitions with valid offsets: %v", offsets) 940 } 941 } 942 return nil 943 }) 944 } 945 946 // TestProducerConsumerTimestamps produces messages with timestamps 947 // and verifies them on consumption. 948 // Requires librdkafka >=0.9.4 and Kafka >=0.10.0.0 949 func TestProducerConsumerTimestamps(t *testing.T) { 950 numver, strver := LibraryVersion() 951 if numver < 0x00090400 { 952 t.Skipf("Requires librdkafka >=0.9.4 (currently on %s)", strver) 953 } 954 955 if !testconfRead() { 956 t.Skipf("Missing testconf.json") 957 } 958 959 consumerConf := ConfigMap{"bootstrap.servers": testconf.Brokers, 960 "go.events.channel.enable": true, 961 "group.id": testconf.Topic, 962 "enable.partition.eof": true, 963 } 964 965 consumerConf.updateFromTestconf() 966 967 /* Create consumer and find recognizable message, verify timestamp. 968 * The consumer is started before the producer to make sure 969 * the message isn't missed. */ 970 t.Logf("Creating consumer") 971 c, err := NewConsumer(&consumerConf) 972 if err != nil { 973 t.Fatalf("NewConsumer: %v", err) 974 } 975 976 t.Logf("Assign %s [0]", testconf.Topic) 977 err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, 978 Offset: OffsetEnd}}) 979 if err != nil { 980 t.Fatalf("Assign: %v", err) 981 } 982 983 /* Wait until EOF is reached so we dont miss the produced message */ 984 for ev := range c.Events() { 985 t.Logf("Awaiting initial EOF") 986 _, ok := ev.(PartitionEOF) 987 if ok { 988 break 989 } 990 } 991 992 /* 993 * Create producer and produce one recognizable message with timestamp 994 */ 995 producerConf := ConfigMap{"bootstrap.servers": testconf.Brokers} 996 producerConf.updateFromTestconf() 997 998 t.Logf("Creating producer") 999 p, err := NewProducer(&producerConf) 1000 if err != nil { 1001 t.Fatalf("NewProducer: %v", err) 1002 } 1003 1004 drChan := make(chan Event, 1) 1005 1006 /* Offset the timestamp to avoid comparison with system clock */ 1007 future, _ := time.ParseDuration("87658h") // 10y 1008 timestamp := time.Now().Add(future) 1009 key := fmt.Sprintf("TS: %v", timestamp) 1010 t.Logf("Producing message with timestamp %v", timestamp) 1011 err = p.Produce(&Message{ 1012 TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 1013 Key: []byte(key), 1014 Timestamp: timestamp}, 1015 drChan) 1016 1017 if err != nil { 1018 t.Fatalf("Produce: %v", err) 1019 } 1020 1021 // Wait for delivery 1022 t.Logf("Awaiting delivery report") 1023 ev := <-drChan 1024 m, ok := ev.(*Message) 1025 if !ok { 1026 t.Fatalf("drChan: Expected *Message, got %v", ev) 1027 } 1028 if m.TopicPartition.Error != nil { 1029 t.Fatalf("Delivery failed: %v", m.TopicPartition) 1030 } 1031 t.Logf("Produced message to %v", m.TopicPartition) 1032 producedOffset := m.TopicPartition.Offset 1033 1034 p.Close() 1035 1036 /* Now consume messages, waiting for that recognizable one. */ 1037 t.Logf("Consuming messages") 1038 outer: 1039 for ev := range c.Events() { 1040 switch m := ev.(type) { 1041 case *Message: 1042 if m.TopicPartition.Error != nil { 1043 continue 1044 } 1045 if m.Key == nil || string(m.Key) != key { 1046 continue 1047 } 1048 1049 t.Logf("Found message at %v with timestamp %s %s", 1050 m.TopicPartition, 1051 m.TimestampType, m.Timestamp) 1052 1053 if m.TopicPartition.Offset != producedOffset { 1054 t.Fatalf("Produced Offset %d does not match consumed offset %d", producedOffset, m.TopicPartition.Offset) 1055 } 1056 1057 if m.TimestampType != TimestampCreateTime { 1058 t.Fatalf("Expected timestamp CreateTime, not %s", 1059 m.TimestampType) 1060 } 1061 1062 /* Since Kafka timestamps are milliseconds we need to 1063 * shave off some precision for the comparison */ 1064 if m.Timestamp.UnixNano()/1000000 != 1065 timestamp.UnixNano()/1000000 { 1066 t.Fatalf("Expected timestamp %v (%d), not %v (%d)", 1067 timestamp, timestamp.UnixNano(), 1068 m.Timestamp, m.Timestamp.UnixNano()) 1069 } 1070 break outer 1071 default: 1072 } 1073 } 1074 1075 c.Close() 1076 } 1077 1078 // TestProducerConsumerHeaders produces messages with headers 1079 // and verifies them on consumption. 1080 // Requires librdkafka >=0.11.4 and Kafka >=0.11.0.0 1081 func TestProducerConsumerHeaders(t *testing.T) { 1082 numver, strver := LibraryVersion() 1083 if numver < 0x000b0400 { 1084 t.Skipf("Requires librdkafka >=0.11.4 (currently on %s, 0x%x)", strver, numver) 1085 } 1086 1087 if !testconfRead() { 1088 t.Skipf("Missing testconf.json") 1089 } 1090 1091 conf := ConfigMap{"bootstrap.servers": testconf.Brokers, 1092 "api.version.request": true, 1093 "enable.auto.commit": false, 1094 "group.id": testconf.Topic, 1095 } 1096 1097 conf.updateFromTestconf() 1098 1099 /* 1100 * Create producer and produce a couple of messages with and without 1101 * headers. 1102 */ 1103 t.Logf("Creating producer") 1104 p, err := NewProducer(&conf) 1105 if err != nil { 1106 t.Fatalf("NewProducer: %v", err) 1107 } 1108 1109 drChan := make(chan Event, 1) 1110 1111 // prepare some header values 1112 bigBytes := make([]byte, 2500) 1113 for i := 0; i < len(bigBytes); i++ { 1114 bigBytes[i] = byte(i) 1115 } 1116 1117 myVarint := make([]byte, binary.MaxVarintLen64) 1118 myVarintLen := binary.PutVarint(myVarint, 12345678901234) 1119 1120 expMsgHeaders := [][]Header{ 1121 { 1122 {"msgid", []byte("1")}, 1123 {"a key with SPACES ", bigBytes[:15]}, 1124 {"BIGONE!", bigBytes}, 1125 }, 1126 { 1127 {"msgid", []byte("2")}, 1128 {"myVarint", myVarint[:myVarintLen]}, 1129 {"empty", []byte("")}, 1130 {"theNullIsNil", nil}, 1131 }, 1132 nil, // no headers 1133 { 1134 {"msgid", []byte("4")}, 1135 {"order", []byte("1")}, 1136 {"order", []byte("2")}, 1137 {"order", nil}, 1138 {"order", []byte("4")}, 1139 }, 1140 } 1141 1142 t.Logf("Producing %d messages", len(expMsgHeaders)) 1143 for _, hdrs := range expMsgHeaders { 1144 err = p.Produce(&Message{ 1145 TopicPartition: TopicPartition{Topic: &testconf.Topic, Partition: 0}, 1146 Headers: hdrs}, 1147 drChan) 1148 } 1149 1150 if err != nil { 1151 t.Fatalf("Produce: %v", err) 1152 } 1153 1154 var firstOffset Offset = OffsetInvalid 1155 for range expMsgHeaders { 1156 ev := <-drChan 1157 m, ok := ev.(*Message) 1158 if !ok { 1159 t.Fatalf("drChan: Expected *Message, got %v", ev) 1160 } 1161 if m.TopicPartition.Error != nil { 1162 t.Fatalf("Delivery failed: %v", m.TopicPartition) 1163 } 1164 t.Logf("Produced message to %v", m.TopicPartition) 1165 if firstOffset == OffsetInvalid { 1166 firstOffset = m.TopicPartition.Offset 1167 } 1168 } 1169 1170 p.Close() 1171 1172 /* Now consume the produced messages and verify the headers */ 1173 t.Logf("Creating consumer starting at offset %v", firstOffset) 1174 c, err := NewConsumer(&conf) 1175 if err != nil { 1176 t.Fatalf("NewConsumer: %v", err) 1177 } 1178 1179 err = c.Assign([]TopicPartition{{Topic: &testconf.Topic, Partition: 0, 1180 Offset: firstOffset}}) 1181 if err != nil { 1182 t.Fatalf("Assign: %v", err) 1183 } 1184 1185 for n, hdrs := range expMsgHeaders { 1186 m, err := c.ReadMessage(-1) 1187 if err != nil { 1188 t.Fatalf("Expected message #%d, not error %v", n, err) 1189 } 1190 1191 if m.Headers == nil { 1192 if hdrs == nil { 1193 continue 1194 } 1195 t.Fatalf("Expected message #%d to have headers", n) 1196 } 1197 1198 if hdrs == nil { 1199 t.Fatalf("Expected message #%d not to have headers, but found %v", n, m.Headers) 1200 } 1201 1202 // Compare headers 1203 if !reflect.DeepEqual(hdrs, m.Headers) { 1204 t.Fatalf("Expected message #%d headers to match %v, but found %v", n, hdrs, m.Headers) 1205 } 1206 1207 t.Logf("Message #%d headers matched: %v", n, m.Headers) 1208 } 1209 1210 c.Close() 1211 } 1212 1213 func validateTopicResult(t *testing.T, result []TopicResult, expError map[string]Error) { 1214 for _, res := range result { 1215 exp, ok := expError[res.Topic] 1216 if !ok { 1217 t.Errorf("Result for unexpected topic %s", res) 1218 continue 1219 } 1220 1221 if res.Error.Code() != exp.Code() { 1222 t.Errorf("Topic %s: expected \"%s\", got \"%s\"", 1223 res.Topic, exp, res.Error) 1224 continue 1225 } 1226 1227 t.Logf("Topic %s: matched expected \"%s\"", res.Topic, res.Error) 1228 } 1229 } 1230 1231 func TestAdminTopics(t *testing.T) { 1232 rand.Seed(time.Now().Unix()) 1233 1234 a := createAdminClient(t) 1235 defer a.Close() 1236 1237 brokerList, err := getBrokerList(a) 1238 if err != nil { 1239 t.Fatalf("Failed to retrieve broker list: %v", err) 1240 } 1241 1242 // Few and Many replica sets use in these tests 1243 var fewReplicas []int32 1244 if len(brokerList) < 2 { 1245 fewReplicas = brokerList 1246 } else { 1247 fewReplicas = brokerList[0:2] 1248 } 1249 1250 var manyReplicas []int32 1251 if len(brokerList) < 5 { 1252 manyReplicas = brokerList 1253 } else { 1254 manyReplicas = brokerList[0:5] 1255 } 1256 1257 const topicCnt = 7 1258 newTopics := make([]TopicSpecification, topicCnt) 1259 1260 expError := map[string]Error{} 1261 1262 for i := 0; i < topicCnt; i++ { 1263 topic := fmt.Sprintf("%s-create-%d-%d", testconf.Topic, i, rand.Intn(100000)) 1264 newTopics[i] = TopicSpecification{ 1265 Topic: topic, 1266 NumPartitions: 1 + i*2, 1267 } 1268 1269 if (i % 1) == 0 { 1270 newTopics[i].ReplicationFactor = len(fewReplicas) 1271 } else { 1272 newTopics[i].ReplicationFactor = len(manyReplicas) 1273 } 1274 1275 expError[newTopics[i].Topic] = Error{} // No error 1276 1277 var useReplicas []int32 1278 if i == 2 { 1279 useReplicas = fewReplicas 1280 } else if i == 3 { 1281 useReplicas = manyReplicas 1282 } else if i == topicCnt-1 { 1283 newTopics[i].ReplicationFactor = len(brokerList) + 10 1284 expError[newTopics[i].Topic] = Error{code: ErrInvalidReplicationFactor} 1285 } 1286 1287 if len(useReplicas) > 0 { 1288 newTopics[i].ReplicaAssignment = make([][]int32, newTopics[i].NumPartitions) 1289 newTopics[i].ReplicationFactor = 0 1290 for p := 0; p < newTopics[i].NumPartitions; p++ { 1291 newTopics[i].ReplicaAssignment[p] = useReplicas 1292 } 1293 } 1294 } 1295 1296 maxDuration, err := time.ParseDuration("30s") 1297 if err != nil { 1298 t.Fatalf("%s", err) 1299 } 1300 1301 // First just validate the topics, don't create 1302 t.Logf("Validating topics before creation\n") 1303 ctx, cancel := context.WithTimeout(context.Background(), maxDuration) 1304 defer cancel() 1305 result, err := a.CreateTopics(ctx, newTopics, 1306 SetAdminValidateOnly(true)) 1307 if err != nil { 1308 t.Fatalf("CreateTopics(ValidateOnly) failed: %s", err) 1309 } 1310 1311 validateTopicResult(t, result, expError) 1312 1313 // Now create the topics 1314 t.Logf("Creating topics\n") 1315 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1316 defer cancel() 1317 result, err = a.CreateTopics(ctx, newTopics, SetAdminValidateOnly(false)) 1318 if err != nil { 1319 t.Fatalf("CreateTopics() failed: %s", err) 1320 } 1321 1322 validateTopicResult(t, result, expError) 1323 1324 // Attempt to create the topics again, should all fail. 1325 t.Logf("Attempt to re-create topics, should all fail\n") 1326 for k := range expError { 1327 if expError[k].code == ErrNoError { 1328 expError[k] = Error{code: ErrTopicAlreadyExists} 1329 } 1330 } 1331 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1332 defer cancel() 1333 result, err = a.CreateTopics(ctx, newTopics) 1334 if err != nil { 1335 t.Fatalf("CreateTopics#2() failed: %s", err) 1336 } 1337 1338 validateTopicResult(t, result, expError) 1339 1340 // Add partitions to some of the topics 1341 t.Logf("Create new partitions for a subset of topics\n") 1342 newParts := make([]PartitionsSpecification, topicCnt/2) 1343 expError = map[string]Error{} 1344 for i := 0; i < topicCnt/2; i++ { 1345 topic := newTopics[i].Topic 1346 newParts[i] = PartitionsSpecification{ 1347 Topic: topic, 1348 IncreaseTo: newTopics[i].NumPartitions + 3, 1349 } 1350 if i == 1 { 1351 // Invalid partition count (less than current) 1352 newParts[i].IncreaseTo = newTopics[i].NumPartitions - 1 1353 expError[topic] = Error{code: ErrInvalidPartitions} 1354 } else { 1355 expError[topic] = Error{} 1356 } 1357 t.Logf("Creating new partitions for %s: %d -> %d: expecting %v\n", 1358 topic, newTopics[i].NumPartitions, newParts[i].IncreaseTo, expError[topic]) 1359 } 1360 1361 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1362 defer cancel() 1363 result, err = a.CreatePartitions(ctx, newParts) 1364 if err != nil { 1365 t.Fatalf("CreatePartitions() failed: %s", err) 1366 } 1367 1368 validateTopicResult(t, result, expError) 1369 1370 // FIXME: wait for topics to become available in metadata instead 1371 time.Sleep(5000 * time.Millisecond) 1372 1373 // Delete the topics 1374 deleteTopics := make([]string, topicCnt) 1375 for i := 0; i < topicCnt; i++ { 1376 deleteTopics[i] = newTopics[i].Topic 1377 if i == topicCnt-1 { 1378 expError[deleteTopics[i]] = Error{code: ErrUnknownTopicOrPart} 1379 } else { 1380 expError[deleteTopics[i]] = Error{} 1381 } 1382 } 1383 1384 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1385 defer cancel() 1386 result2, err := a.DeleteTopics(ctx, deleteTopics) 1387 if err != nil { 1388 t.Fatalf("DeleteTopics() failed: %s", err) 1389 } 1390 1391 validateTopicResult(t, result2, expError) 1392 } 1393 1394 func validateConfig(t *testing.T, results []ConfigResourceResult, expResults []ConfigResourceResult, checkConfigEntries bool) { 1395 1396 _, file, line, _ := runtime.Caller(1) 1397 caller := fmt.Sprintf("%s:%d", path.Base(file), line) 1398 1399 if len(results) != len(expResults) { 1400 t.Fatalf("%s: Expected %d results, got %d: %v", caller, len(expResults), len(results), results) 1401 } 1402 1403 for i, result := range results { 1404 expResult := expResults[i] 1405 1406 if result.Error.Code() != expResult.Error.Code() { 1407 t.Errorf("%s: %v: Expected %v, got %v", caller, result, expResult.Error.Code(), result.Error.Code()) 1408 continue 1409 } 1410 1411 if !checkConfigEntries { 1412 continue 1413 } 1414 1415 matchCnt := 0 1416 for _, expEntry := range expResult.Config { 1417 1418 entry, ok := result.Config[expEntry.Name] 1419 if !ok { 1420 t.Errorf("%s: %v: expected config %s not found in result", caller, result, expEntry.Name) 1421 continue 1422 } 1423 1424 if entry.Value != expEntry.Value { 1425 t.Errorf("%s: %v: expected config %s to have value \"%s\", not \"%s\"", caller, result, expEntry.Name, expEntry.Value, entry.Value) 1426 continue 1427 } 1428 1429 matchCnt++ 1430 } 1431 1432 if matchCnt != len(expResult.Config) { 1433 t.Errorf("%s: %v: only %d/%d expected configs matched", caller, result, matchCnt, len(expResult.Config)) 1434 } 1435 } 1436 1437 if t.Failed() { 1438 t.Fatalf("%s: ConfigResourceResult validation failed: see previous errors", caller) 1439 } 1440 } 1441 1442 func TestAdminConfig(t *testing.T) { 1443 rand.Seed(time.Now().Unix()) 1444 1445 a := createAdminClient(t) 1446 defer a.Close() 1447 1448 // Steps: 1449 // 1) Create a topic, providing initial non-default configuration 1450 // 2) Read back config to verify 1451 // 3) Alter config 1452 // 4) Read back config to verify 1453 // 5) Delete the topic 1454 1455 topic := fmt.Sprintf("%s-config-%d", testconf.Topic, rand.Intn(100000)) 1456 1457 // Expected config 1458 expResources := []ConfigResourceResult{ 1459 { 1460 Type: ResourceTopic, 1461 Name: topic, 1462 Config: map[string]ConfigEntryResult{ 1463 "compression.type": ConfigEntryResult{ 1464 Name: "compression.type", 1465 Value: "snappy", 1466 }, 1467 }, 1468 }, 1469 } 1470 // Create topic 1471 newTopics := []TopicSpecification{{ 1472 Topic: topic, 1473 NumPartitions: 1, 1474 ReplicationFactor: 1, 1475 Config: map[string]string{"compression.type": "snappy"}, 1476 }} 1477 1478 ctx, cancel := context.WithCancel(context.Background()) 1479 defer cancel() 1480 topicResult, err := a.CreateTopics(ctx, newTopics) 1481 if err != nil { 1482 t.Fatalf("Create topic request failed: %v", err) 1483 } 1484 1485 if topicResult[0].Error.Code() != ErrNoError { 1486 t.Fatalf("Failed to create topic %s: %s", topic, topicResult[0].Error) 1487 } 1488 1489 // Wait for topic to show up in metadata before performing 1490 // subsequent operations on it, otherwise we risk DescribeConfigs() 1491 // to fail with UnknownTopic.. (this is really a broker issue). 1492 // Sometimes even the metadata is not enough, so we add an 1493 // arbitrary 10s sleep too. 1494 t.Logf("Waiting for new topic %s to show up in metadata and stabilize", topic) 1495 err = waitTopicInMetadata(a, topic, 10*1000) // 10s 1496 if err != nil { 1497 t.Fatalf("%v", err) 1498 } 1499 t.Logf("Topic %s now in metadata, waiting another 10s for stabilization", topic) 1500 time.Sleep(10 * time.Second) 1501 1502 // Read back config to validate 1503 configResources := []ConfigResource{{Type: ResourceTopic, Name: topic}} 1504 describeRes, err := a.DescribeConfigs(ctx, configResources) 1505 if err != nil { 1506 t.Fatalf("Describe configs request failed: %v", err) 1507 } 1508 1509 validateConfig(t, describeRes, expResources, true) 1510 1511 // Alter some configs. 1512 // Configuration alterations are currently atomic, all values 1513 // need to be passed, otherwise non-passed values will be reverted 1514 // to their default values. 1515 // Future versions will allow incremental updates: 1516 // https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API 1517 newConfig := make(map[string]string) 1518 for _, entry := range describeRes[0].Config { 1519 newConfig[entry.Name] = entry.Value 1520 } 1521 1522 // Change something 1523 newConfig["retention.ms"] = "86400000" 1524 newConfig["message.timestamp.type"] = "LogAppendTime" 1525 1526 for k, v := range newConfig { 1527 expResources[0].Config[k] = ConfigEntryResult{Name: k, Value: v} 1528 } 1529 1530 configResources = []ConfigResource{{Type: ResourceTopic, Name: topic, Config: StringMapToConfigEntries(newConfig, AlterOperationSet)}} 1531 alterRes, err := a.AlterConfigs(ctx, configResources) 1532 if err != nil { 1533 t.Fatalf("Alter configs request failed: %v", err) 1534 } 1535 1536 validateConfig(t, alterRes, expResources, false) 1537 1538 // Read back config to validate 1539 configResources = []ConfigResource{{Type: ResourceTopic, Name: topic}} 1540 describeRes, err = a.DescribeConfigs(ctx, configResources) 1541 if err != nil { 1542 t.Fatalf("Describe configs request failed: %v", err) 1543 } 1544 1545 validateConfig(t, describeRes, expResources, true) 1546 1547 // Delete the topic 1548 // FIXME: wait for topics to become available in metadata instead 1549 time.Sleep(5000 * time.Millisecond) 1550 1551 topicResult, err = a.DeleteTopics(ctx, []string{topic}) 1552 if err != nil { 1553 t.Fatalf("DeleteTopics() failed: %s", err) 1554 } 1555 1556 if topicResult[0].Error.Code() != ErrNoError { 1557 t.Fatalf("Failed to delete topic %s: %s", topic, topicResult[0].Error) 1558 } 1559 1560 } 1561 1562 //Test AdminClient GetMetadata API 1563 func TestAdminGetMetadata(t *testing.T) { 1564 if !testconfRead() { 1565 t.Skipf("Missing testconf.json") 1566 } 1567 1568 config := &ConfigMap{"bootstrap.servers": testconf.Brokers} 1569 config.updateFromTestconf() 1570 1571 // Create Admin client 1572 a, err := NewAdminClient(config) 1573 if err != nil { 1574 t.Errorf("Failed to create Admin client: %s\n", err) 1575 return 1576 } 1577 defer a.Close() 1578 1579 metaData, err := a.GetMetadata(&testconf.Topic, false, 5*1000) 1580 if err != nil { 1581 t.Errorf("Failed to get meta data for topic %s. Error: %s\n", testconf.Topic, err) 1582 return 1583 } 1584 t.Logf("Meta data for topic %s: %v\n", testconf.Topic, metaData) 1585 1586 metaData, err = a.GetMetadata(nil, true, 5*1000) 1587 if err != nil { 1588 t.Errorf("Failed to get meta data, Error: %s\n", err) 1589 return 1590 } 1591 t.Logf("Meta data for admin client: %v\n", metaData) 1592 1593 } 1594 1595 // Test AdminClient ClusterID. 1596 func TestAdminClient_ClusterID(t *testing.T) { 1597 if !testconfRead() { 1598 t.Skipf("Missing testconf.json") 1599 } 1600 1601 config := &ConfigMap{"bootstrap.servers": testconf.Brokers} 1602 if err := config.updateFromTestconf(); err != nil { 1603 t.Fatalf("Failed to update test configuration: %s\n", err) 1604 } 1605 1606 admin, err := NewAdminClient(config) 1607 if err != nil { 1608 t.Fatalf("Failed to create Admin client: %s\n", err) 1609 } 1610 defer admin.Close() 1611 1612 ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) 1613 defer cancel() 1614 clusterID, err := admin.ClusterID(ctx) 1615 if err != nil { 1616 t.Fatalf("Failed to get ClusterID: %s\n", err) 1617 } 1618 if clusterID == "" { 1619 t.Fatal("ClusterID is empty.") 1620 } 1621 1622 t.Logf("ClusterID: %s\n", clusterID) 1623 } 1624 1625 // Test AdminClient ControllerID. 1626 func TestAdminClient_ControllerID(t *testing.T) { 1627 if !testconfRead() { 1628 t.Skipf("Missing testconf.json") 1629 } 1630 1631 config := &ConfigMap{"bootstrap.servers": testconf.Brokers} 1632 if err := config.updateFromTestconf(); err != nil { 1633 t.Fatalf("Failed to update test configuration: %s\n", err) 1634 } 1635 1636 producer, err := NewProducer(config) 1637 if err != nil { 1638 t.Fatalf("Failed to create Producer client: %s\n", err) 1639 } 1640 admin, err := NewAdminClientFromProducer(producer) 1641 if err != nil { 1642 t.Fatalf("Failed to create Admin client: %s\n", err) 1643 } 1644 defer admin.Close() 1645 1646 ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) 1647 defer cancel() 1648 controllerID, err := admin.ControllerID(ctx) 1649 if err != nil { 1650 t.Fatalf("Failed to get ControllerID: %s\n", err) 1651 } 1652 if controllerID < 0 { 1653 t.Fatalf("ControllerID is negative: %d\n", controllerID) 1654 } 1655 1656 t.Logf("ControllerID: %d\n", controllerID) 1657 } 1658 1659 func TestAdminACLs(t *testing.T) { 1660 if !testconfRead() { 1661 t.Skipf("Missing testconf.json") 1662 } 1663 1664 rand.Seed(time.Now().Unix()) 1665 topic := testconf.Topic 1666 group := testconf.GroupID 1667 noError := NewError(ErrNoError, "", false) 1668 unknownError := NewError(ErrUnknown, "Unknown broker error", false) 1669 var expectedCreateACLs []CreateACLResult 1670 var expectedDescribeACLs DescribeACLsResult 1671 var expectedDeleteACLs []DeleteACLsResult 1672 var ctx context.Context 1673 var cancel context.CancelFunc 1674 1675 a := createAdminClient(t) 1676 defer a.Close() 1677 1678 maxDuration, err := time.ParseDuration("30s") 1679 if err != nil { 1680 t.Fatalf("%s", err) 1681 } 1682 requestTimeout, err := time.ParseDuration("20s") 1683 if err != nil { 1684 t.Fatalf("%s", err) 1685 } 1686 1687 checkExpectedResult := func(expected interface{}, result interface{}) { 1688 if !reflect.DeepEqual(result, expected) { 1689 t.Fatalf("Expected result to deep equal to %v, but found %v", expected, result) 1690 } 1691 } 1692 1693 // Create ACLs 1694 t.Logf("Creating ACLs\n") 1695 newACLs := ACLBindings{ 1696 { 1697 Type: ResourceTopic, 1698 Name: topic, 1699 ResourcePatternType: ResourcePatternTypeLiteral, 1700 Principal: "User:test-user-1", 1701 Host: "*", 1702 Operation: ACLOperationRead, 1703 PermissionType: ACLPermissionTypeAllow, 1704 }, 1705 { 1706 Type: ResourceTopic, 1707 Name: topic, 1708 ResourcePatternType: ResourcePatternTypePrefixed, 1709 Principal: "User:test-user-2", 1710 Host: "*", 1711 Operation: ACLOperationWrite, 1712 PermissionType: ACLPermissionTypeDeny, 1713 }, 1714 { 1715 Type: ResourceGroup, 1716 Name: group, 1717 ResourcePatternType: ResourcePatternTypePrefixed, 1718 Principal: "User:test-user-2", 1719 Host: "*", 1720 Operation: ACLOperationAll, 1721 PermissionType: ACLPermissionTypeAllow, 1722 }, 1723 } 1724 1725 invalidACLs := ACLBindings{ 1726 { 1727 Type: ResourceTopic, 1728 Name: topic, 1729 ResourcePatternType: ResourcePatternTypeLiteral, 1730 // Principal must be in the form "{principalType}:{principalName}" 1731 // Broker returns ErrUnknown in this case 1732 Principal: "wrong-principal", 1733 Host: "*", 1734 Operation: ACLOperationRead, 1735 PermissionType: ACLPermissionTypeAllow, 1736 }, 1737 } 1738 1739 aclBindingFilters := ACLBindingFilters{ 1740 { 1741 Type: ResourceAny, 1742 ResourcePatternType: ResourcePatternTypeAny, 1743 Operation: ACLOperationAny, 1744 PermissionType: ACLPermissionTypeAny, 1745 }, 1746 { 1747 Type: ResourceAny, 1748 ResourcePatternType: ResourcePatternTypePrefixed, 1749 Operation: ACLOperationAny, 1750 PermissionType: ACLPermissionTypeAny, 1751 }, 1752 { 1753 Type: ResourceTopic, 1754 ResourcePatternType: ResourcePatternTypeAny, 1755 Operation: ACLOperationAny, 1756 PermissionType: ACLPermissionTypeAny, 1757 }, 1758 { 1759 Type: ResourceGroup, 1760 ResourcePatternType: ResourcePatternTypeAny, 1761 Operation: ACLOperationAny, 1762 PermissionType: ACLPermissionTypeAny, 1763 }, 1764 } 1765 1766 // CreateACLs should be idempotent 1767 for n := 0; n < 2; n++ { 1768 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1769 defer cancel() 1770 1771 resultCreateACLs, err := a.CreateACLs(ctx, newACLs, SetAdminRequestTimeout(requestTimeout)) 1772 if err != nil { 1773 t.Fatalf("CreateACLs() failed: %s", err) 1774 } 1775 expectedCreateACLs = []CreateACLResult{{Error: noError}, {Error: noError}, {Error: noError}} 1776 checkExpectedResult(expectedCreateACLs, resultCreateACLs) 1777 } 1778 1779 // CreateACLs with server side validation errors 1780 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1781 defer cancel() 1782 1783 resultCreateACLs, err := a.CreateACLs(ctx, invalidACLs, SetAdminRequestTimeout(requestTimeout)) 1784 if err != nil { 1785 t.Fatalf("CreateACLs() failed: %s", err) 1786 } 1787 expectedCreateACLs = []CreateACLResult{{Error: unknownError}} 1788 checkExpectedResult(expectedCreateACLs, resultCreateACLs) 1789 1790 // DescribeACLs must return the three ACLs 1791 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1792 defer cancel() 1793 resultDescribeACLs, err := a.DescribeACLs(ctx, aclBindingFilters[0], SetAdminRequestTimeout(requestTimeout)) 1794 expectedDescribeACLs = DescribeACLsResult{ 1795 Error: noError, 1796 ACLBindings: newACLs, 1797 } 1798 if err != nil { 1799 t.Fatalf("%s", err) 1800 } 1801 sort.Sort(&resultDescribeACLs.ACLBindings) 1802 checkExpectedResult(expectedDescribeACLs, *resultDescribeACLs) 1803 1804 // Delete the ACLs with ResourcePatternTypePrefixed 1805 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1806 defer cancel() 1807 resultDeleteACLs, err := a.DeleteACLs(ctx, aclBindingFilters[1:2], SetAdminRequestTimeout(requestTimeout)) 1808 expectedDeleteACLs = []DeleteACLsResult{ 1809 { 1810 Error: noError, 1811 ACLBindings: newACLs[1:3], 1812 }, 1813 } 1814 if err != nil { 1815 t.Fatalf("%s", err) 1816 } 1817 sort.Sort(&resultDeleteACLs[0].ACLBindings) 1818 checkExpectedResult(expectedDeleteACLs, resultDeleteACLs) 1819 1820 // Delete the ACLs with ResourceTopic and ResourceGroup 1821 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1822 defer cancel() 1823 resultDeleteACLs, err = a.DeleteACLs(ctx, aclBindingFilters[2:4], SetAdminRequestTimeout(requestTimeout)) 1824 expectedDeleteACLs = []DeleteACLsResult{ 1825 { 1826 Error: noError, 1827 ACLBindings: newACLs[0:1], 1828 }, 1829 { 1830 Error: noError, 1831 ACLBindings: ACLBindings{}, 1832 }, 1833 } 1834 if err != nil { 1835 t.Fatalf("%s", err) 1836 } 1837 checkExpectedResult(expectedDeleteACLs, resultDeleteACLs) 1838 1839 // All the ACLs should have been deleted 1840 ctx, cancel = context.WithTimeout(context.Background(), maxDuration) 1841 defer cancel() 1842 resultDescribeACLs, err = a.DescribeACLs(ctx, aclBindingFilters[0], SetAdminRequestTimeout(requestTimeout)) 1843 expectedDescribeACLs = DescribeACLsResult{ 1844 Error: noError, 1845 ACLBindings: ACLBindings{}, 1846 } 1847 if err != nil { 1848 t.Fatalf("%s", err) 1849 } 1850 checkExpectedResult(expectedDescribeACLs, *resultDescribeACLs) 1851 }