github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/consumer_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 "fmt" 21 "os" 22 "reflect" 23 "sort" 24 "strings" 25 "sync" 26 "sync/atomic" 27 "testing" 28 "time" 29 ) 30 31 // TestConsumerAPIs dry-tests most Consumer APIs, no broker is needed. 32 func TestConsumerAPIs(t *testing.T) { 33 34 c, err := NewConsumer(&ConfigMap{}) 35 if err == nil { 36 t.Fatalf("Expected NewConsumer() to fail without group.id") 37 } 38 39 c, err = NewConsumer(&ConfigMap{ 40 "group.id": "gotest", 41 "socket.timeout.ms": 10, 42 "session.timeout.ms": 10, 43 "enable.auto.offset.store": false, // permit StoreOffsets() 44 }) 45 if err != nil { 46 t.Fatalf("%s", err) 47 } 48 49 t.Logf("Consumer %s", c) 50 51 err = c.Subscribe("gotest", nil) 52 if err != nil { 53 t.Errorf("Subscribe failed: %s", err) 54 } 55 56 err = c.SubscribeTopics([]string{"gotest1", "gotest2"}, 57 func(my_c *Consumer, ev Event) error { 58 t.Logf("%s", ev) 59 return nil 60 }) 61 if err != nil { 62 t.Errorf("SubscribeTopics failed: %s", err) 63 } 64 65 _, err = c.Commit() 66 if err != nil && err.(Error).Code() != ErrNoOffset { 67 t.Errorf("Commit() failed: %s", err) 68 } 69 70 err = c.Unsubscribe() 71 if err != nil { 72 t.Errorf("Unsubscribe failed: %s", err) 73 } 74 75 topic := "gotest" 76 stored, err := c.StoreOffsets([]TopicPartition{{Topic: &topic, Partition: 0, Offset: 1}}) 77 if err != nil && err.(Error).Code() != ErrUnknownPartition { 78 t.Errorf("StoreOffsets() failed: %s", err) 79 toppar := stored[0] 80 if toppar.Error != nil && toppar.Error.(Error).Code() == ErrUnknownPartition { 81 t.Errorf("StoreOffsets() TopicPartition error: %s", toppar.Error) 82 } 83 } 84 var empty []TopicPartition 85 stored, err = c.StoreOffsets(empty) 86 if err != nil { 87 t.Errorf("StoreOffsets(empty) failed: %s", err) 88 } 89 90 // test StoreMessage doesn't fail either 91 stored, err = c.StoreMessage(&Message{TopicPartition: TopicPartition{Topic: &topic, Partition: 0, Offset: 1}}) 92 if err != nil && err.(Error).Code() != ErrUnknownPartition { 93 t.Errorf("StoreMessage() failed: %s", err) 94 toppar := stored[0] 95 if toppar.Error != nil && toppar.Error.(Error).Code() == ErrUnknownPartition { 96 t.Errorf("StoreMessage() TopicPartition error: %s", toppar.Error) 97 } 98 } 99 100 topic1 := "gotest1" 101 topic2 := "gotest2" 102 err = c.Assign([]TopicPartition{{Topic: &topic1, Partition: 2}, 103 {Topic: &topic2, Partition: 1}}) 104 if err != nil { 105 t.Errorf("Assign failed: %s", err) 106 } 107 108 err = c.Seek(TopicPartition{Topic: &topic1, Partition: 2, Offset: -1}, 1000) 109 if err != nil { 110 t.Errorf("Seek failed: %s", err) 111 } 112 113 // Pause & Resume 114 err = c.Pause([]TopicPartition{{Topic: &topic1, Partition: 2}, 115 {Topic: &topic2, Partition: 1}}) 116 if err != nil { 117 t.Errorf("Pause failed: %s", err) 118 } 119 err = c.Resume([]TopicPartition{{Topic: &topic1, Partition: 2}, 120 {Topic: &topic2, Partition: 1}}) 121 if err != nil { 122 t.Errorf("Resume failed: %s", err) 123 } 124 125 err = c.Unassign() 126 if err != nil { 127 t.Errorf("Unassign failed: %s", err) 128 } 129 130 // Incremental Assign & Unassign 131 err = c.IncrementalAssign([]TopicPartition{ 132 {Topic: &topic1, Partition: 9, Offset: 1}, 133 {Topic: &topic2, Partition: 40, Offset: OffsetEnd}, 134 {Topic: &topic1, Partition: 10, Offset: OffsetInvalid}, 135 {Topic: &topic2, Partition: 30}, 136 }) 137 if err != nil { 138 t.Errorf("IncrementalAssign failed: %s", err) 139 } 140 141 err = c.IncrementalUnassign([]TopicPartition{ 142 {Topic: &topic2, Partition: 30}, 143 {Topic: &topic2, Partition: 40}, 144 {Topic: &topic1, Partition: 10}, 145 }) 146 if err != nil { 147 t.Errorf("IncrementalUnassign failed: %s", err) 148 } 149 150 assignment, err := c.Assignment() 151 if err != nil { 152 t.Errorf("Assignment (after incremental) failed: %s", err) 153 } 154 155 t.Logf("(Incremental) Assignment: %s\n", assignment) 156 if len(assignment) != 1 || 157 *assignment[0].Topic != topic1 || 158 assignment[0].Partition != 9 { 159 t.Errorf("(Incremental) Assignment mismatch: %v", assignment) 160 } 161 162 // ConsumerGroupMetadata 163 _, err = c.GetConsumerGroupMetadata() 164 if err != nil { 165 t.Errorf("Expected valid ConsumerGroupMetadata: %v", err) 166 } 167 168 _, err = NewTestConsumerGroupMetadata("mygroup") 169 if err != nil { 170 t.Errorf("Expected valid ConsumerGroupMetadata: %v", err) 171 } 172 173 topic = "mytopic" 174 // OffsetsForTimes 175 offsets, err := c.OffsetsForTimes([]TopicPartition{{Topic: &topic, Offset: 12345}}, 100) 176 t.Logf("OffsetsForTimes() returned Offsets %s and error %s\n", offsets, err) 177 if err == nil { 178 t.Errorf("OffsetsForTimes() should have failed\n") 179 } 180 if offsets != nil { 181 t.Errorf("OffsetsForTimes() failed but returned non-nil Offsets: %s\n", offsets) 182 } 183 184 // Position 185 offsets, err = c.Position([]TopicPartition{ 186 {Topic: &topic, Partition: 10}, 187 {Topic: &topic, Partition: 5}, 188 }) 189 t.Logf("Position() returned Offsets %s and error %v\n", offsets, err) 190 if err != nil { 191 t.Errorf("Position() should not have failed\n") 192 } 193 if offsets == nil { 194 t.Errorf("Position() should not have returned nil\n") 195 } 196 197 // Committed 198 offsets, err = c.Committed([]TopicPartition{{Topic: &topic, Partition: 5}}, 10) 199 t.Logf("Committed() returned Offsets %s and error %s\n", offsets, err) 200 if err == nil { 201 t.Errorf("Committed() should have failed\n") 202 } 203 if offsets != nil { 204 t.Errorf("Committed() failed but returned non-nil Offsets: %s\n", offsets) 205 } 206 207 err = c.Close() 208 if err != nil { 209 t.Errorf("Close failed: %s", err) 210 } 211 } 212 213 func TestConsumerSubscription(t *testing.T) { 214 c, err := NewConsumer(&ConfigMap{"group.id": "gotest"}) 215 if err != nil { 216 t.Fatalf("%s", err) 217 } 218 219 topics := []string{"gotest1", "gotest2", "gotest3"} 220 sort.Strings(topics) 221 222 err = c.SubscribeTopics(topics, nil) 223 if err != nil { 224 t.Fatalf("SubscribeTopics failed: %s", err) 225 } 226 227 subscription, err := c.Subscription() 228 if err != nil { 229 t.Fatalf("Subscription() failed: %s", err) 230 } 231 232 sort.Strings(subscription) 233 234 t.Logf("Compare Subscription %v to original list of topics %v\n", 235 subscription, topics) 236 237 r := reflect.DeepEqual(topics, subscription) 238 if r != true { 239 t.Fatalf("Subscription() %v does not match original topics %v", 240 subscription, topics) 241 } 242 c.Close() 243 } 244 245 func TestConsumerAssignment(t *testing.T) { 246 c, err := NewConsumer(&ConfigMap{"group.id": "gotest"}) 247 if err != nil { 248 t.Fatalf("%s", err) 249 } 250 251 topic0 := "topic0" 252 topic1 := "topic1" 253 partitions := TopicPartitions{ 254 {Topic: &topic1, Partition: 1}, 255 {Topic: &topic1, Partition: 3}, 256 {Topic: &topic0, Partition: 2}} 257 sort.Sort(partitions) 258 259 err = c.Assign(partitions) 260 if err != nil { 261 t.Fatalf("Assign failed: %s", err) 262 } 263 264 assignment, err := c.Assignment() 265 if err != nil { 266 t.Fatalf("Assignment() failed: %s", err) 267 } 268 269 sort.Sort(TopicPartitions(assignment)) 270 271 t.Logf("Compare Assignment %v to original list of partitions %v\n", 272 assignment, partitions) 273 274 // Use Logf instead of Errorf for timeout-checking errors on CI builds 275 // since CI environments are unreliable timing-wise. 276 tmoutFunc := t.Errorf 277 _, onCi := os.LookupEnv("CI") 278 if onCi { 279 tmoutFunc = t.Logf 280 } 281 282 // Test ReadMessage() 283 for _, tmout := range []time.Duration{0, 200 * time.Millisecond} { 284 start := time.Now() 285 m, err := c.ReadMessage(tmout) 286 duration := time.Since(start) 287 288 t.Logf("ReadMessage(%v) ret %v and %v in %v", tmout, m, err, duration) 289 if m != nil || err == nil { 290 t.Errorf("Expected ReadMessage to fail: %v, %v", m, err) 291 } 292 if err.(Error).Code() != ErrTimedOut { 293 t.Errorf("Expected ReadMessage to fail with ErrTimedOut, not %v", err) 294 } 295 296 if tmout == 0 { 297 if duration.Seconds() > 0.1 { 298 tmoutFunc("Expected ReadMessage(%v) to fail after max 100ms, not %v", tmout, duration) 299 } 300 } else if tmout > 0 { 301 if duration.Seconds() < tmout.Seconds()*0.75 || duration.Seconds() > tmout.Seconds()*1.25 { 302 tmoutFunc("Expected ReadMessage() to fail after %v -+25%%, not %v", tmout, duration) 303 } 304 } 305 } 306 307 // reflect.DeepEqual() can't be used since TopicPartition.Topic 308 // is a pointer to a string rather than a string and the pointer 309 // will differ between partitions and assignment. 310 // Instead do a simple stringification + string compare. 311 if fmt.Sprintf("%v", assignment) != fmt.Sprintf("%v", partitions) { 312 t.Fatalf("Assignment() %v does not match original partitions %v", 313 assignment, partitions) 314 } 315 c.Close() 316 } 317 318 func TestConsumerOAuthBearerConfig(t *testing.T) { 319 myOAuthConfig := "scope=myscope principal=gotest" 320 321 c, err := NewConsumer(&ConfigMap{ 322 "group.id": "test", 323 "security.protocol": "SASL_PLAINTEXT", 324 "go.events.channel.enable": true, 325 "sasl.mechanisms": "OAUTHBEARER", 326 "sasl.oauthbearer.config": myOAuthConfig, 327 }) 328 if err != nil { 329 t.Fatalf("NewConsumer failed: %s", err) 330 } 331 332 // Wait for initial OAuthBearerTokenRefresh and check 333 // that its SerializerConfig string is identical to myOAuthConfig 334 for { 335 ev := <-c.Events() 336 oatr, ok := ev.(OAuthBearerTokenRefresh) 337 if !ok { 338 continue 339 } 340 341 t.Logf("Got %s with SerializerConfig \"%s\"", oatr, oatr.Config) 342 343 if oatr.Config != myOAuthConfig { 344 t.Fatalf("%s: Expected .SerializerConfig to be %s, not %s", 345 oatr, myOAuthConfig, oatr.Config) 346 } 347 348 // Verify that we can set a token 349 err = c.SetOAuthBearerToken(OAuthBearerToken{ 350 TokenValue: "aaaa", 351 Expiration: time.Now().Add(time.Second * time.Duration(60)), 352 Principal: "gotest", 353 }) 354 if err != nil { 355 t.Fatalf("Failed to set token: %s", err) 356 } 357 358 // Verify that we can set a token refresh failure 359 err = c.SetOAuthBearerTokenFailure("A token failure test") 360 if err != nil { 361 t.Fatalf("Failed to set token failure: %s", err) 362 } 363 364 break 365 } 366 367 c.Close() 368 } 369 370 func TestConsumerLog(t *testing.T) { 371 logsChan := make(chan LogEvent, 1000) 372 373 c, err := NewConsumer(&ConfigMap{ 374 "debug": "all", 375 "go.logs.channel.enable": true, 376 "go.logs.channel": logsChan, 377 "group.id": "gotest"}) 378 if err != nil { 379 t.Fatalf("%s", err) 380 } 381 382 if c.Logs() != logsChan { 383 t.Fatalf("Expected c.Logs() %v == logsChan %v", c.Logs(), logsChan) 384 } 385 386 expectedLogs := map[struct { 387 tag string 388 message string 389 }]bool{ 390 {"MEMBERID", "gotest"}: false, 391 {"CGRPSTATE", "gotest"}: false, 392 {"CGRPQUERY", "gotest"}: false, 393 } 394 395 go func() { 396 for { 397 select { 398 case log, ok := <-logsChan: 399 if !ok { 400 return 401 } 402 403 for expectedLog, found := range expectedLogs { 404 if found { 405 continue 406 } 407 if log.Tag != expectedLog.tag { 408 continue 409 } 410 if strings.Contains(log.Message, expectedLog.message) { 411 expectedLogs[expectedLog] = true 412 } 413 } 414 } 415 } 416 }() 417 418 <-time.After(time.Second * 3) 419 420 if err := c.Close(); err != nil { 421 t.Fatal("Failed to close consumer.") 422 } 423 424 for expectedLog, found := range expectedLogs { 425 if !found { 426 t.Errorf( 427 "Expected to find log with tag `%s' and message containing `%s',"+ 428 " but didn't find any.", 429 expectedLog.tag, 430 expectedLog.message) 431 } 432 } 433 } 434 435 func wrapRebalanceCb(assignedEvents *int32, revokedEvents *int32, t *testing.T) func(c *Consumer, event Event) error { 436 return func(c *Consumer, event Event) error { 437 switch ev := event.(type) { 438 case AssignedPartitions: 439 atomic.AddInt32(assignedEvents, 1) 440 441 t.Logf("%v, %s rebalance: %d new partition(s) assigned: %v\n", 442 c, c.GetRebalanceProtocol(), len(ev.Partitions), 443 ev.Partitions) 444 err := c.Assign(ev.Partitions) 445 if err != nil { 446 panic(err) 447 } 448 449 case RevokedPartitions: 450 atomic.AddInt32(revokedEvents, 1) 451 452 t.Logf("%v, %s rebalance: %d partition(s) revoked: %v\n", 453 c, c.GetRebalanceProtocol(), len(ev.Partitions), 454 ev.Partitions) 455 if c.AssignmentLost() { 456 // Our consumer has been kicked out of the group and the 457 // entire assignment is thus lost. 458 t.Logf("%v, Current assignment lost!\n", c) 459 } 460 461 // The client automatically calls Unassign() unless 462 // the callback has already called that method. 463 } 464 return nil 465 } 466 } 467 468 func testPoll(c *Consumer, doneChan chan bool, t *testing.T, wg *sync.WaitGroup) { 469 defer wg.Done() 470 471 run := true 472 for run { 473 select { 474 case <-doneChan: 475 run = false 476 477 default: 478 ev := c.Poll(100) 479 if ev == nil { 480 continue 481 } 482 switch e := ev.(type) { 483 case *Message: 484 t.Logf("Message on %s:\n%s\n", 485 e.TopicPartition, string(e.Value)) 486 if e.Headers != nil { 487 t.Logf("Headers: %v\n", e.Headers) 488 } 489 490 case Error: 491 // Errors should generally be 492 // considered informational, the client 493 // will try to automatically recover. 494 t.Logf("Error: %v: %v for "+ 495 "consumer %v\n", e.Code(), e, c) 496 497 default: 498 t.Logf("Ignored %v for consumer %v\n", 499 e, c) 500 } 501 } 502 } 503 } 504 505 // TestConsumerCloseForStaticMember verifies the rebalance 506 // for static membership. 507 // According to KIP-345, the consumer group will not trigger rebalance unless 508 // 1) A new member joins 509 // 2) A leader rejoins (possibly due to topic assignment change) 510 // 3) An existing member offline time is over session timeout 511 // 4) Broker receives a leave group request containing alistof 512 // `group.instance.id`s (details later) 513 // 514 // This test uses 3 consumers while each consumer joins after the assignment 515 // finished for the previous consumers. 516 // The expected behavior for these consumers are: 517 // 1) First consumer joins, AssignedPartitions happens. Assign all the 518 // partitions to it. 519 // 2) Second consumer joins, RevokedPartitions happens from the first consumer, 520 // then AssignedPartitions happens to both consumers. 521 // 3) Third consumer joins, RevokedPartitions happens from the previous two 522 // consumers, then AssignedPartitions happens to all the three consumers. 523 // 4) Close the second consumer, revoke its assignments will happen, but it 524 // should not notice other consumers. 525 // 5) Rejoin the second consumer, rebalance should not happen to all the other 526 // consumers since it's not the leader, AssignedPartitions only happened 527 // to this consumer to assign the partitions. 528 // 6) Close the third consumer, revoke its assignments will happen, but it 529 // should not notice other consumers. 530 // 7) Close the rejoined consumer, revoke its assignments will happen, 531 // but it should not notice other consumers. 532 // 8) Close the first consumer, revoke its assignments will happen. 533 // 534 // The total number of AssignedPartitions for the first consumer is 3, 535 // and the total number of RevokedPartitions for the first consumer is 3. 536 // The total number of AssignedPartitions for the second consumer is 2, 537 // and the total number of RevokedPartitions for the second consumer is 2. 538 // The total number of AssignedPartitions for the third consumer is 1, 539 // and the total number of RevokedPartitions for the third consumer is 1. 540 // The total number of AssignedPartitions for the rejoined consumer 541 // (originally second consumer) is 1, 542 // and the total number of RevokedPartitions for the rejoined consumer 543 // (originally second consumer) is 1. 544 func TestConsumerCloseForStaticMember(t *testing.T) { 545 if !testconfRead() { 546 t.Skipf("Missing testconf.json") 547 } 548 broker := testconf.Brokers 549 topic := createTestTopic(t, "staticMembership", 3, 1) 550 551 var assignedEvents1 int32 552 var revokedEvents1 int32 553 554 var assignedEvents2 int32 555 var revokedEvents2 int32 556 557 var assignedEvents3 int32 558 var revokedEvents3 int32 559 560 var assignedEvents4 int32 561 var revokedEvents4 int32 562 563 conf1 := ConfigMap{ 564 "bootstrap.servers": broker, 565 "group.id": "rebalance", 566 "session.timeout.ms": "6000", 567 "max.poll.interval.ms": "10000", 568 "group.instance.id": "staticmember1", 569 } 570 c1, err := NewConsumer(&conf1) 571 572 conf2 := ConfigMap{ 573 "bootstrap.servers": broker, 574 "group.id": "rebalance", 575 "session.timeout.ms": "6000", 576 "max.poll.interval.ms": "10000", 577 "group.instance.id": "staticmember2", 578 } 579 c2, err := NewConsumer(&conf2) 580 if err != nil { 581 t.Fatalf("%s", err) 582 } 583 584 conf3 := ConfigMap{ 585 "bootstrap.servers": broker, 586 "group.id": "rebalance", 587 "session.timeout.ms": "6000", 588 "max.poll.interval.ms": "10000", 589 "group.instance.id": "staticmember3", 590 } 591 592 c3, err := NewConsumer(&conf3) 593 if err != nil { 594 t.Fatalf("%s", err) 595 } 596 wrapRebalancecb1 := wrapRebalanceCb(&assignedEvents1, &revokedEvents1, t) 597 err = c1.Subscribe(topic, wrapRebalancecb1) 598 if err != nil { 599 t.Fatalf("Failed to subscribe to topic %s: %s\n", topic, err) 600 } 601 602 wg := sync.WaitGroup{} 603 doneChan := make(chan bool, 3) 604 605 wg.Add(1) 606 go testPoll(c1, doneChan, t, &wg) 607 testConsumerWaitAssignment(c1, t) 608 609 closeChan := make(chan bool) 610 wrapRebalancecb2 := wrapRebalanceCb(&assignedEvents2, &revokedEvents2, t) 611 err = c2.Subscribe(topic, wrapRebalancecb2) 612 if err != nil { 613 t.Fatalf("Failed to subscribe to topic %s: %s\n", topic, err) 614 } 615 wg.Add(1) 616 go testPoll(c2, closeChan, t, &wg) 617 testConsumerWaitAssignment(c2, t) 618 619 wrapRebalancecb3 := wrapRebalanceCb(&assignedEvents3, &revokedEvents3, t) 620 err = c3.Subscribe(topic, wrapRebalancecb3) 621 if err != nil { 622 t.Fatalf("Failed to subscribe to topic %s: %s\n", topic, err) 623 } 624 wg.Add(1) 625 go testPoll(c3, doneChan, t, &wg) 626 testConsumerWaitAssignment(c3, t) 627 628 closeChan <- true 629 close(closeChan) 630 c2.Close() 631 632 c2, err = NewConsumer(&conf2) 633 if err != nil { 634 t.Fatalf("%s", err) 635 } 636 637 wrapRebalancecb4 := wrapRebalanceCb(&assignedEvents4, &revokedEvents4, t) 638 err = c2.Subscribe(topic, wrapRebalancecb4) 639 if err != nil { 640 t.Fatalf("Failed to subscribe to topic %s: %s\n", topic, err) 641 } 642 643 wg.Add(1) 644 go testPoll(c2, doneChan, t, &wg) 645 testConsumerWaitAssignment(c2, t) 646 647 doneChan <- true 648 close(doneChan) 649 650 c3.Close() 651 c2.Close() 652 c1.Close() 653 654 wg.Wait() 655 656 // Wait 2 * session.timeout.ms to make sure no revokedEvents happens 657 time.Sleep(2 * 6000 * time.Millisecond) 658 659 if atomic.LoadInt32(&assignedEvents1) != 3 { 660 t.Fatalf("3 assignedEvents are Expected to happen for the first consumer, but %d happened\n", 661 atomic.LoadInt32(&assignedEvents1)) 662 } 663 664 if atomic.LoadInt32(&revokedEvents1) != 3 { 665 t.Fatalf("3 revokedEvents are Expected to happen for the first consumer, but %d happened\n", 666 atomic.LoadInt32(&revokedEvents1)) 667 } 668 669 if atomic.LoadInt32(&assignedEvents2) != 2 { 670 t.Fatalf("2 assignedEvents are Expected to happen for the second consumer, but %d happened\n", 671 atomic.LoadInt32(&assignedEvents2)) 672 } 673 if atomic.LoadInt32(&revokedEvents2) != 2 { 674 t.Fatalf("2 revokedEvents is Expected to happen for the second consumer, but %d happened\n", 675 atomic.LoadInt32(&revokedEvents2)) 676 } 677 678 if atomic.LoadInt32(&assignedEvents3) != 1 { 679 t.Fatalf("1 assignedEvents is Expected to happen for the third consumer, but %d happened\n", 680 atomic.LoadInt32(&assignedEvents3)) 681 } 682 if atomic.LoadInt32(&revokedEvents3) != 1 { 683 t.Fatalf("1 revokedEvents is Expected to happen for the third consumer, but %d happened\n", 684 atomic.LoadInt32(&revokedEvents3)) 685 } 686 687 if atomic.LoadInt32(&assignedEvents4) != 1 { 688 t.Fatalf("1 assignedEvents is Expected to happen for the rejoined consumer(originally second consumer), but %d happened\n", 689 atomic.LoadInt32(&assignedEvents4)) 690 } 691 if atomic.LoadInt32(&revokedEvents4) != 1 { 692 t.Fatalf("1 revokedEvents is Expected to happen for the rejoined consumer(originally second consumer), but %d happened\n", 693 atomic.LoadInt32(&revokedEvents4)) 694 } 695 } 696 697 func testConsumerWaitAssignment(c *Consumer, t *testing.T) { 698 run := true 699 for run { 700 assignment, err := c.Assignment() 701 if err != nil { 702 t.Fatalf("Assignment failed: %s\n", err) 703 } 704 705 if len(assignment) != 0 { 706 t.Logf("%v Assigned partitions are: %v\n", c, assignment) 707 run = false 708 } 709 } 710 }