forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy path0113-cooperative_rebalance.cpp
3440 lines (2777 loc) · 120 KB
/
0113-cooperative_rebalance.cpp
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2020-2022, Magnus Edenhill
* 2023, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <iostream>
#include <map>
#include <set>
#include <algorithm>
#include <cstring>
#include <cstdlib>
#include <assert.h>
#include "testcpp.h"
#include <fstream>
extern "C" {
#include "../src/rdkafka_protocol.h"
#include "test.h"
}
using namespace std;
/** Topic+Partition helper class */
class Toppar {
public:
Toppar(const string &topic, int32_t partition) : partition(partition) {
this->topic.append(topic);
}
Toppar(const RdKafka::TopicPartition *tp) :
topic(tp->topic()), partition(tp->partition()) {
}
friend bool operator==(const Toppar &a, const Toppar &b) {
return a.partition == b.partition && a.topic == b.topic;
}
friend bool operator<(const Toppar &a, const Toppar &b) {
if (a.partition < b.partition)
return true;
return a.topic < b.topic;
}
string str() const {
return tostr() << topic << "[" << partition << "]";
}
std::string topic;
int32_t partition;
};
static std::string get_bootstrap_servers() {
RdKafka::Conf *conf;
std::string bootstrap_servers;
Test::conf_init(&conf, NULL, 0);
conf->get("bootstrap.servers", bootstrap_servers);
delete conf;
return bootstrap_servers;
}
class DrCb : public RdKafka::DeliveryReportCb {
public:
void dr_cb(RdKafka::Message &msg) {
if (msg.err())
Test::Fail("Delivery failed: " + RdKafka::err2str(msg.err()));
}
};
/**
* @brief Produce messages to partitions.
*
* The pair is Toppar,msg_cnt_per_partition.
* The Toppar is topic,partition_cnt.
*/
static void produce_msgs(vector<pair<Toppar, int> > partitions) {
RdKafka::Conf *conf;
Test::conf_init(&conf, NULL, 0);
string errstr;
DrCb dr;
conf->set("dr_cb", &dr, errstr);
RdKafka::Producer *p = RdKafka::Producer::create(conf, errstr);
if (!p)
Test::Fail("Failed to create producer: " + errstr);
delete conf;
for (vector<pair<Toppar, int> >::iterator it = partitions.begin();
it != partitions.end(); it++) {
for (int part = 0; part < it->first.partition; part++) {
for (int i = 0; i < it->second; i++) {
RdKafka::ErrorCode err =
p->produce(it->first.topic, part, RdKafka::Producer::RK_MSG_COPY,
(void *)"Hello there", 11, NULL, 0, 0, NULL);
TEST_ASSERT(!err, "produce(%s, %d) failed: %s", it->first.topic.c_str(),
part, RdKafka::err2str(err).c_str());
p->poll(0);
}
}
}
p->flush(10000);
delete p;
}
static RdKafka::KafkaConsumer *make_consumer(
string client_id,
string group_id,
string assignment_strategy,
vector<pair<string, string> > *additional_conf,
RdKafka::RebalanceCb *rebalance_cb,
int timeout_s) {
std::string bootstraps;
std::string errstr;
std::vector<std::pair<std::string, std::string> >::iterator itr;
RdKafka::Conf *conf;
Test::conf_init(&conf, NULL, timeout_s);
Test::conf_set(conf, "client.id", client_id);
Test::conf_set(conf, "group.id", group_id);
Test::conf_set(conf, "auto.offset.reset", "earliest");
Test::conf_set(conf, "enable.auto.commit", "false");
Test::conf_set(conf, "partition.assignment.strategy", assignment_strategy);
if (test_consumer_group_protocol()) {
Test::conf_set(conf, "group.protocol", test_consumer_group_protocol());
}
if (additional_conf != NULL) {
for (itr = (*additional_conf).begin(); itr != (*additional_conf).end();
itr++)
Test::conf_set(conf, itr->first, itr->second);
}
if (rebalance_cb) {
if (conf->set("rebalance_cb", rebalance_cb, errstr))
Test::Fail("Failed to set rebalance_cb: " + errstr);
}
RdKafka::KafkaConsumer *consumer =
RdKafka::KafkaConsumer::create(conf, errstr);
if (!consumer)
Test::Fail("Failed to create KafkaConsumer: " + errstr);
delete conf;
return consumer;
}
/**
* @returns a CSV string of the vector
*/
static string string_vec_to_str(const vector<string> &v) {
ostringstream ss;
for (vector<string>::const_iterator it = v.begin(); it != v.end(); it++)
ss << (it == v.begin() ? "" : ", ") << *it;
return ss.str();
}
void expect_assignment(RdKafka::KafkaConsumer *consumer, size_t count) {
std::vector<RdKafka::TopicPartition *> partitions;
RdKafka::ErrorCode err;
err = consumer->assignment(partitions);
if (err)
Test::Fail(consumer->name() +
" assignment() failed: " + RdKafka::err2str(err));
if (partitions.size() != count)
Test::Fail(tostr() << "Expecting consumer " << consumer->name()
<< " to have " << count
<< " assigned partition(s), not: " << partitions.size());
RdKafka::TopicPartition::destroy(partitions);
}
static bool TopicPartition_cmp(const RdKafka::TopicPartition *a,
const RdKafka::TopicPartition *b) {
if (a->topic() < b->topic())
return true;
else if (a->topic() > b->topic())
return false;
return a->partition() < b->partition();
}
void expect_assignment(RdKafka::KafkaConsumer *consumer,
vector<RdKafka::TopicPartition *> &expected) {
vector<RdKafka::TopicPartition *> partitions;
RdKafka::ErrorCode err;
err = consumer->assignment(partitions);
if (err)
Test::Fail(consumer->name() +
" assignment() failed: " + RdKafka::err2str(err));
if (partitions.size() != expected.size())
Test::Fail(tostr() << "Expecting consumer " << consumer->name()
<< " to have " << expected.size()
<< " assigned partition(s), not " << partitions.size());
sort(partitions.begin(), partitions.end(), TopicPartition_cmp);
sort(expected.begin(), expected.end(), TopicPartition_cmp);
int fails = 0;
for (int i = 0; i < (int)partitions.size(); i++) {
if (!TopicPartition_cmp(partitions[i], expected[i]))
continue;
Test::Say(tostr() << _C_RED << consumer->name() << ": expected assignment #"
<< i << " " << expected[i]->topic() << " ["
<< expected[i]->partition() << "], not "
<< partitions[i]->topic() << " ["
<< partitions[i]->partition() << "]\n");
fails++;
}
if (fails)
Test::Fail(consumer->name() + ": Expected assignment mismatch, see above");
RdKafka::TopicPartition::destroy(partitions);
}
class DefaultRebalanceCb : public RdKafka::RebalanceCb {
private:
static string part_list_print(
const vector<RdKafka::TopicPartition *> &partitions) {
ostringstream ss;
for (unsigned int i = 0; i < partitions.size(); i++)
ss << (i == 0 ? "" : ", ") << partitions[i]->topic() << " ["
<< partitions[i]->partition() << "]";
return ss.str();
}
public:
int assign_call_cnt;
int revoke_call_cnt;
int nonempty_assign_call_cnt; /**< ASSIGN_PARTITIONS with partitions */
int lost_call_cnt;
int partitions_assigned_net;
bool wait_rebalance;
int64_t ts_last_assign; /**< Timestamp of last rebalance assignment */
map<Toppar, int> msg_cnt; /**< Number of consumed messages per partition. */
~DefaultRebalanceCb() {
reset_msg_cnt();
}
DefaultRebalanceCb() :
assign_call_cnt(0),
revoke_call_cnt(0),
nonempty_assign_call_cnt(0),
lost_call_cnt(0),
partitions_assigned_net(0),
wait_rebalance(false),
ts_last_assign(0) {
}
void rebalance_cb(RdKafka::KafkaConsumer *consumer,
RdKafka::ErrorCode err,
std::vector<RdKafka::TopicPartition *> &partitions) {
wait_rebalance = false;
std::string protocol = consumer->rebalance_protocol();
if (protocol != "") {
/* Consumer hasn't been closed */
TEST_ASSERT(protocol == "COOPERATIVE",
"%s: Expected rebalance_protocol \"COOPERATIVE\", not %s",
consumer->name().c_str(), protocol.c_str());
}
const char *lost_str = consumer->assignment_lost() ? " (LOST)" : "";
Test::Say(tostr() << _C_YEL "RebalanceCb " << protocol << ": "
<< consumer->name() << " " << RdKafka::err2str(err)
<< lost_str << ": " << part_list_print(partitions)
<< "\n");
if (err == RdKafka::ERR__ASSIGN_PARTITIONS) {
if (consumer->assignment_lost())
Test::Fail("unexpected lost assignment during ASSIGN rebalance");
RdKafka::Error *error = consumer->incremental_assign(partitions);
if (error)
Test::Fail(tostr() << "consumer->incremental_assign() failed: "
<< error->str());
if (partitions.size() > 0)
nonempty_assign_call_cnt++;
assign_call_cnt += 1;
partitions_assigned_net += (int)partitions.size();
ts_last_assign = test_clock();
} else {
if (consumer->assignment_lost())
lost_call_cnt += 1;
RdKafka::Error *error = consumer->incremental_unassign(partitions);
if (error)
Test::Fail(tostr() << "consumer->incremental_unassign() failed: "
<< error->str());
if (partitions.size() == 0)
Test::Fail("revoked partitions size should never be 0");
revoke_call_cnt += 1;
partitions_assigned_net -= (int)partitions.size();
}
/* Reset message counters for the given partitions. */
Test::Say(consumer->name() + ": resetting message counters:\n");
reset_msg_cnt(partitions);
}
bool poll_once(RdKafka::KafkaConsumer *c, int timeout_ms) {
RdKafka::Message *msg = c->consume(timeout_ms);
bool ret = msg->err() != RdKafka::ERR__TIMED_OUT;
if (!msg->err())
msg_cnt[Toppar(msg->topic_name(), msg->partition())]++;
delete msg;
return ret;
}
void reset_msg_cnt() {
msg_cnt.clear();
}
void reset_msg_cnt(Toppar &tp) {
int msgcnt = get_msg_cnt(tp);
Test::Say(tostr() << " RESET " << tp.topic << " [" << tp.partition << "]"
<< " with " << msgcnt << " messages\n");
if (!msg_cnt.erase(tp) && msgcnt)
Test::Fail("erase failed!");
}
void reset_msg_cnt(const vector<RdKafka::TopicPartition *> &partitions) {
for (unsigned int i = 0; i < partitions.size(); i++) {
Toppar tp(partitions[i]->topic(), partitions[i]->partition());
reset_msg_cnt(tp);
}
}
int get_msg_cnt(const Toppar &tp) {
map<Toppar, int>::iterator it = msg_cnt.find(tp);
if (it == msg_cnt.end())
return 0;
return it->second;
}
};
/**
* @brief Verify that the consumer's assignment is a subset of the
* subscribed topics.
*
* @param allow_mismatch Allow assignment of not subscribed topics.
* This can happen when the subscription is updated
* but a rebalance callback hasn't been seen yet.
* @param all_assignments Accumulated assignments for all consumers.
* If an assigned partition already exists it means
* the partition is assigned to multiple consumers and
* the test will fail.
* @param exp_msg_cnt Expected message count per assigned partition, or -1
* if not to check.
*
* @returns the number of assigned partitions, or fails if the
* assignment is empty or there is an assignment for
* topic that is not subscribed.
*/
static int verify_consumer_assignment(
RdKafka::KafkaConsumer *consumer,
DefaultRebalanceCb &rebalance_cb,
const vector<string> &topics,
bool allow_empty,
bool allow_mismatch,
map<Toppar, RdKafka::KafkaConsumer *> *all_assignments,
int exp_msg_cnt) {
vector<RdKafka::TopicPartition *> partitions;
RdKafka::ErrorCode err;
int fails = 0;
int count;
ostringstream ss;
err = consumer->assignment(partitions);
TEST_ASSERT(!err, "Failed to get assignment for consumer %s: %s",
consumer->name().c_str(), RdKafka::err2str(err).c_str());
count = (int)partitions.size();
for (vector<RdKafka::TopicPartition *>::iterator it = partitions.begin();
it != partitions.end(); it++) {
RdKafka::TopicPartition *p = *it;
if (find(topics.begin(), topics.end(), p->topic()) == topics.end()) {
Test::Say(tostr() << (allow_mismatch ? _C_YEL "Warning (allowed)"
: _C_RED "Error")
<< ": " << consumer->name() << " is assigned "
<< p->topic() << " [" << p->partition() << "] which is "
<< "not in the list of subscribed topics: "
<< string_vec_to_str(topics) << "\n");
if (!allow_mismatch)
fails++;
}
Toppar tp(p);
pair<map<Toppar, RdKafka::KafkaConsumer *>::iterator, bool> ret;
ret = all_assignments->insert(
pair<Toppar, RdKafka::KafkaConsumer *>(tp, consumer));
if (!ret.second) {
Test::Say(tostr() << _C_RED << "Error: " << consumer->name()
<< " is assigned " << p->topic() << " ["
<< p->partition()
<< "] which is "
"already assigned to consumer "
<< ret.first->second->name() << "\n");
fails++;
}
int msg_cnt = rebalance_cb.get_msg_cnt(tp);
if (exp_msg_cnt != -1 && msg_cnt != exp_msg_cnt) {
Test::Say(tostr() << _C_RED << "Error: " << consumer->name()
<< " expected " << exp_msg_cnt << " messages on "
<< p->topic() << " [" << p->partition() << "], not "
<< msg_cnt << "\n");
fails++;
}
ss << (it == partitions.begin() ? "" : ", ") << p->topic() << " ["
<< p->partition() << "] (" << msg_cnt << "msgs)";
}
RdKafka::TopicPartition::destroy(partitions);
Test::Say(tostr() << "Consumer " << consumer->name() << " assignment ("
<< count << "): " << ss.str() << "\n");
if (count == 0 && !allow_empty)
Test::Fail("Consumer " + consumer->name() +
" has unexpected empty assignment");
if (fails)
Test::Fail(
tostr() << "Consumer " + consumer->name()
<< " assignment verification failed (see previous error)");
return count;
}
/* -------- a_assign_tests
*
* check behavior incremental assign / unassign outside the context of a
* rebalance.
*/
/** Incremental assign, then assign(NULL).
*/
static void assign_test_1(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2) {
RdKafka::ErrorCode err;
RdKafka::Error *error;
Test::Say("Incremental assign, then assign(NULL)\n");
if ((error = consumer->incremental_assign(toppars1)))
Test::Fail(tostr() << "Incremental assign failed: " << error->str());
Test::check_assignment(consumer, 1, &toppars1[0]->topic());
if ((err = consumer->unassign()))
Test::Fail("Unassign failed: " + RdKafka::err2str(err));
Test::check_assignment(consumer, 0, NULL);
}
/** Assign, then incremental unassign.
*/
static void assign_test_2(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2) {
RdKafka::ErrorCode err;
RdKafka::Error *error;
Test::Say("Assign, then incremental unassign\n");
if ((err = consumer->assign(toppars1)))
Test::Fail("Assign failed: " + RdKafka::err2str(err));
Test::check_assignment(consumer, 1, &toppars1[0]->topic());
if ((error = consumer->incremental_unassign(toppars1)))
Test::Fail("Incremental unassign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
}
/** Incremental assign, then incremental unassign.
*/
static void assign_test_3(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2) {
RdKafka::Error *error;
Test::Say("Incremental assign, then incremental unassign\n");
if ((error = consumer->incremental_assign(toppars1)))
Test::Fail("Incremental assign failed: " + error->str());
Test::check_assignment(consumer, 1, &toppars1[0]->topic());
if ((error = consumer->incremental_unassign(toppars1)))
Test::Fail("Incremental unassign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
}
/** Multi-topic incremental assign and unassign + message consumption.
*/
static void assign_test_4(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2) {
RdKafka::Error *error;
Test::Say(
"Multi-topic incremental assign and unassign + message consumption\n");
if ((error = consumer->incremental_assign(toppars1)))
Test::Fail("Incremental assign failed: " + error->str());
Test::check_assignment(consumer, 1, &toppars1[0]->topic());
RdKafka::Message *m = consumer->consume(5000);
if (m->err() != RdKafka::ERR_NO_ERROR)
Test::Fail("Expecting a consumed message.");
if (m->len() != 100)
Test::Fail(tostr() << "Expecting msg len to be 100, not: "
<< m->len()); /* implies read from topic 1. */
delete m;
if ((error = consumer->incremental_unassign(toppars1)))
Test::Fail("Incremental unassign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
m = consumer->consume(100);
if (m->err() != RdKafka::ERR__TIMED_OUT)
Test::Fail("Not expecting a consumed message.");
delete m;
if ((error = consumer->incremental_assign(toppars2)))
Test::Fail("Incremental assign failed: " + error->str());
Test::check_assignment(consumer, 1, &toppars2[0]->topic());
m = consumer->consume(5000);
if (m->err() != RdKafka::ERR_NO_ERROR)
Test::Fail("Expecting a consumed message.");
if (m->len() != 200)
Test::Fail(tostr() << "Expecting msg len to be 200, not: "
<< m->len()); /* implies read from topic 2. */
delete m;
if ((error = consumer->incremental_assign(toppars1)))
Test::Fail("Incremental assign failed: " + error->str());
if (Test::assignment_partition_count(consumer, NULL) != 2)
Test::Fail(tostr() << "Expecting current assignment to have size 2, not: "
<< Test::assignment_partition_count(consumer, NULL));
m = consumer->consume(5000);
if (m->err() != RdKafka::ERR_NO_ERROR)
Test::Fail("Expecting a consumed message.");
delete m;
if ((error = consumer->incremental_unassign(toppars2)))
Test::Fail("Incremental unassign failed: " + error->str());
if ((error = consumer->incremental_unassign(toppars1)))
Test::Fail("Incremental unassign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
}
/** Incremental assign and unassign of empty collection.
*/
static void assign_test_5(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2) {
RdKafka::Error *error;
std::vector<RdKafka::TopicPartition *> toppars3;
Test::Say("Incremental assign and unassign of empty collection\n");
if ((error = consumer->incremental_assign(toppars3)))
Test::Fail("Incremental assign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
if ((error = consumer->incremental_unassign(toppars3)))
Test::Fail("Incremental unassign failed: " + error->str());
Test::check_assignment(consumer, 0, NULL);
}
static void run_test(
const std::string &t1,
const std::string &t2,
void (*test)(RdKafka::KafkaConsumer *consumer,
std::vector<RdKafka::TopicPartition *> toppars1,
std::vector<RdKafka::TopicPartition *> toppars2)) {
std::vector<RdKafka::TopicPartition *> toppars1;
toppars1.push_back(RdKafka::TopicPartition::create(t1, 0));
std::vector<RdKafka::TopicPartition *> toppars2;
toppars2.push_back(RdKafka::TopicPartition::create(t2, 0));
RdKafka::KafkaConsumer *consumer =
make_consumer("C_1", t1, "cooperative-sticky", NULL, NULL, 10);
test(consumer, toppars1, toppars2);
RdKafka::TopicPartition::destroy(toppars1);
RdKafka::TopicPartition::destroy(toppars2);
consumer->close();
delete consumer;
}
static void a_assign_tests() {
SUB_TEST_QUICK();
int msgcnt = 1000;
const int msgsize1 = 100;
const int msgsize2 = 200;
std::string topic1_str = Test::mk_topic_name("0113-a1", 1);
test_create_topic(NULL, topic1_str.c_str(), 1, 1);
std::string topic2_str = Test::mk_topic_name("0113-a2", 1);
test_create_topic(NULL, topic2_str.c_str(), 1, 1);
test_wait_topic_exists(NULL, topic1_str.c_str(), 10 * 1000);
test_wait_topic_exists(NULL, topic2_str.c_str(), 10 * 1000);
test_produce_msgs_easy_size(topic1_str.c_str(), 0, 0, msgcnt, msgsize1);
test_produce_msgs_easy_size(topic2_str.c_str(), 0, 0, msgcnt, msgsize2);
run_test(topic1_str, topic2_str, assign_test_1);
run_test(topic1_str, topic2_str, assign_test_2);
run_test(topic1_str, topic2_str, assign_test_3);
run_test(topic1_str, topic2_str, assign_test_4);
run_test(topic1_str, topic2_str, assign_test_5);
SUB_TEST_PASS();
}
/**
* @brief Quick Assign 1,2, Assign 2,3, Assign 1,2,3 test to verify
* that the correct OffsetFetch response is used.
* See note in rdkafka_assignment.c for details.
*
* Makes use of the mock cluster to induce latency.
*/
static void a_assign_rapid() {
SUB_TEST_QUICK();
std::string group_id = __FUNCTION__;
rd_kafka_mock_cluster_t *mcluster;
const char *bootstraps;
mcluster = test_mock_cluster_new(3, &bootstraps);
int32_t coord_id = 1;
rd_kafka_mock_coordinator_set(mcluster, "group", group_id.c_str(), coord_id);
rd_kafka_mock_topic_create(mcluster, "topic1", 1, 1);
rd_kafka_mock_topic_create(mcluster, "topic2", 1, 1);
rd_kafka_mock_topic_create(mcluster, "topic3", 1, 1);
/*
* Produce messages to topics
*/
const int msgs_per_partition = 1000;
RdKafka::Conf *pconf;
Test::conf_init(&pconf, NULL, 10);
Test::conf_set(pconf, "bootstrap.servers", bootstraps);
Test::conf_set(pconf, "security.protocol", "plaintext");
std::string errstr;
RdKafka::Producer *p = RdKafka::Producer::create(pconf, errstr);
if (!p)
Test::Fail(tostr() << __FUNCTION__
<< ": Failed to create producer: " << errstr);
delete pconf;
Test::produce_msgs(p, "topic1", 0, msgs_per_partition, 10,
false /*no flush*/);
Test::produce_msgs(p, "topic2", 0, msgs_per_partition, 10,
false /*no flush*/);
Test::produce_msgs(p, "topic3", 0, msgs_per_partition, 10,
false /*no flush*/);
p->flush(10 * 1000);
delete p;
vector<RdKafka::TopicPartition *> toppars1;
toppars1.push_back(RdKafka::TopicPartition::create("topic1", 0));
vector<RdKafka::TopicPartition *> toppars2;
toppars2.push_back(RdKafka::TopicPartition::create("topic2", 0));
vector<RdKafka::TopicPartition *> toppars3;
toppars3.push_back(RdKafka::TopicPartition::create("topic3", 0));
RdKafka::Conf *conf;
Test::conf_init(&conf, NULL, 20);
Test::conf_set(conf, "bootstrap.servers", bootstraps);
Test::conf_set(conf, "security.protocol", "plaintext");
Test::conf_set(conf, "client.id", __FUNCTION__);
Test::conf_set(conf, "group.id", group_id);
Test::conf_set(conf, "auto.offset.reset", "earliest");
Test::conf_set(conf, "enable.auto.commit", "false");
if (test_consumer_group_protocol()) {
Test::conf_set(conf, "group.protocol", test_consumer_group_protocol());
}
RdKafka::KafkaConsumer *consumer;
consumer = RdKafka::KafkaConsumer::create(conf, errstr);
if (!consumer)
Test::Fail(tostr() << __FUNCTION__
<< ": Failed to create consumer: " << errstr);
delete conf;
vector<RdKafka::TopicPartition *> toppars;
vector<RdKafka::TopicPartition *> expected;
map<Toppar, int64_t> pos; /* Expected consume position per partition */
pos[Toppar(toppars1[0]->topic(), toppars1[0]->partition())] = 0;
pos[Toppar(toppars2[0]->topic(), toppars2[0]->partition())] = 0;
pos[Toppar(toppars3[0]->topic(), toppars3[0]->partition())] = 0;
/* To make sure offset commits are fetched in proper assign sequence
* we commit an offset that should not be used in the final consume loop.
* This commit will be overwritten below with another commit. */
vector<RdKafka::TopicPartition *> offsets;
offsets.push_back(RdKafka::TopicPartition::create(
toppars1[0]->topic(), toppars1[0]->partition(), 11));
/* This partition should start at this position even though
* there will be a sub-sequent commit to overwrite it, that should not
* be used since this partition is never unassigned. */
offsets.push_back(RdKafka::TopicPartition::create(
toppars2[0]->topic(), toppars2[0]->partition(), 22));
pos[Toppar(toppars2[0]->topic(), toppars2[0]->partition())] = 22;
Test::print_TopicPartitions("pre-commit", offsets);
RdKafka::ErrorCode err;
err = consumer->commitSync(offsets);
if (err)
Test::Fail(tostr() << __FUNCTION__ << ": pre-commit failed: "
<< RdKafka::err2str(err) << "\n");
/* Add coordinator delay so that the OffsetFetchRequest originating
* from the coming incremental_assign() will not finish before
* we call incremental_unassign() and incremental_assign() again, resulting
* in a situation where the initial OffsetFetchResponse will contain
* an older offset for a previous assignment of one partition. */
rd_kafka_mock_broker_set_rtt(mcluster, coord_id, 5000);
/* Assign 1,2 == 1,2 */
toppars.push_back(toppars1[0]);
toppars.push_back(toppars2[0]);
expected.push_back(toppars1[0]);
expected.push_back(toppars2[0]);
Test::incremental_assign(consumer, toppars);
expect_assignment(consumer, expected);
/* Unassign -1 == 2 */
toppars.clear();
toppars.push_back(toppars1[0]);
vector<RdKafka::TopicPartition *>::iterator it =
find(expected.begin(), expected.end(), toppars1[0]);
expected.erase(it);
Test::incremental_unassign(consumer, toppars);
expect_assignment(consumer, expected);
/* Commit offset for the removed partition and the partition that is
* unchanged in the assignment. */
RdKafka::TopicPartition::destroy(offsets);
offsets.push_back(RdKafka::TopicPartition::create(
toppars1[0]->topic(), toppars1[0]->partition(), 55));
offsets.push_back(RdKafka::TopicPartition::create(
toppars2[0]->topic(), toppars2[0]->partition(), 33)); /* should not be
* used. */
pos[Toppar(toppars1[0]->topic(), toppars1[0]->partition())] = 55;
Test::print_TopicPartitions("commit", offsets);
err = consumer->commitAsync(offsets);
if (err)
Test::Fail(tostr() << __FUNCTION__
<< ": commit failed: " << RdKafka::err2str(err) << "\n");
/* Assign +3 == 2,3 */
toppars.clear();
toppars.push_back(toppars3[0]);
expected.push_back(toppars3[0]);
Test::incremental_assign(consumer, toppars);
expect_assignment(consumer, expected);
/* Now remove the latency */
Test::Say(_C_MAG "Clearing rtt\n");
rd_kafka_mock_broker_set_rtt(mcluster, coord_id, 0);
/* Assign +1 == 1,2,3 */
toppars.clear();
toppars.push_back(toppars1[0]);
expected.push_back(toppars1[0]);
Test::incremental_assign(consumer, toppars);
expect_assignment(consumer, expected);
/*
* Verify consumed messages
*/
int wait_end = (int)expected.size();
while (wait_end > 0) {
RdKafka::Message *msg = consumer->consume(10 * 1000);
if (msg->err() == RdKafka::ERR__TIMED_OUT)
Test::Fail(tostr() << __FUNCTION__
<< ": Consume timed out waiting "
"for "
<< wait_end << " more partitions");
Toppar tp = Toppar(msg->topic_name(), msg->partition());
int64_t *exp_pos = &pos[tp];
Test::Say(3, tostr() << __FUNCTION__ << ": Received " << tp.topic << " ["
<< tp.partition << "] at offset " << msg->offset()
<< " (expected offset " << *exp_pos << ")\n");
if (*exp_pos != msg->offset())
Test::Fail(tostr() << __FUNCTION__ << ": expected message offset "
<< *exp_pos << " for " << msg->topic_name() << " ["
<< msg->partition() << "], not " << msg->offset()
<< "\n");
(*exp_pos)++;
if (*exp_pos == msgs_per_partition) {
TEST_ASSERT(wait_end > 0, "");
wait_end--;
} else if (msg->offset() > msgs_per_partition)
Test::Fail(tostr() << __FUNCTION__ << ": unexpected message with "
<< "offset " << msg->offset() << " on " << tp.topic
<< " [" << tp.partition << "]\n");
delete msg;
}
RdKafka::TopicPartition::destroy(offsets);
RdKafka::TopicPartition::destroy(toppars1);
RdKafka::TopicPartition::destroy(toppars2);
RdKafka::TopicPartition::destroy(toppars3);
delete consumer;
test_mock_cluster_destroy(mcluster);
SUB_TEST_PASS();
}
/* Check behavior when:
* 1. single topic with 2 partitions.
* 2. consumer 1 (with rebalance_cb) subscribes to it.
* 3. consumer 2 (with rebalance_cb) subscribes to it.
* 4. close.
*/
static void b_subscribe_with_cb_test(rd_bool_t close_consumer) {
SUB_TEST("%s", close_consumer ? "close consumer" : "don't close consumer");
int expected_cb1_assign_call_cnt = 3;
int expected_cb2_assign_call_cnt = 2;
std::string topic_name = Test::mk_topic_name("0113-cooperative_rebalance", 1);
std::string group_name =
Test::mk_unique_group_name("0113-cooperative_rebalance");
test_create_topic(NULL, topic_name.c_str(), 2, 1);
DefaultRebalanceCb rebalance_cb1;
RdKafka::KafkaConsumer *c1 = make_consumer(
"C_1", group_name, "cooperative-sticky", NULL, &rebalance_cb1, 25);
DefaultRebalanceCb rebalance_cb2;
RdKafka::KafkaConsumer *c2 = make_consumer(
"C_2", group_name, "cooperative-sticky", NULL, &rebalance_cb2, 25);
test_wait_topic_exists(c1->c_ptr(), topic_name.c_str(), 10 * 1000);
Test::subscribe(c1, topic_name);
bool c2_subscribed = false;
while (true) {
Test::poll_once(c1, 500);
Test::poll_once(c2, 500);
/* Start c2 after c1 has received initial assignment */
if (!c2_subscribed && rebalance_cb1.nonempty_assign_call_cnt > 0) {
Test::subscribe(c2, topic_name);
c2_subscribed = true;
}
/* Failure case: test will time out. */
if (Test::assignment_partition_count(c1, NULL) == 1 &&
Test::assignment_partition_count(c2, NULL) == 1) {
/* Callback count can vary in KIP-848 */
if (test_consumer_group_protocol_classic() &&
!(rebalance_cb1.assign_call_cnt == expected_cb1_assign_call_cnt &&
rebalance_cb2.assign_call_cnt == expected_cb2_assign_call_cnt))
continue;
break;
}
}
/* Sequence of events:
*
* 1. c1 joins group.
* 2. c1 gets assigned 2 partitions (+1 assign call).
* - there isn't a follow-on rebalance because there aren't any revoked
* partitions.
* 3. c2 joins group.
* 4. This results in a rebalance with one partition being revoked from c1 (+1
* revoke call), and no partitions assigned to either c1 (+1 assign call) or
* c2 (+1 assign call) (however the rebalance callback will be called in each
* case with an empty set).
* 5. c1 then re-joins the group since it had a partition revoked.
* 6. c2 is now assigned a single partition (+1 assign call), and c1's
* incremental assignment is empty (+1 assign call).
* 7. Since there were no revoked partitions, no further rebalance is
* triggered.
*/
/* Callback count can vary in KIP-848 */
if (test_consumer_group_protocol_classic()) {
/* The rebalance cb is always called on assign, even if empty. */
if (rebalance_cb1.assign_call_cnt != expected_cb1_assign_call_cnt)
Test::Fail(tostr() << "Expecting " << expected_cb1_assign_call_cnt
<< " assign calls on consumer 1, not "
<< rebalance_cb1.assign_call_cnt);
if (rebalance_cb2.assign_call_cnt != expected_cb2_assign_call_cnt)
Test::Fail(tostr() << "Expecting " << expected_cb2_assign_call_cnt
<< " assign calls on consumer 2, not: "
<< rebalance_cb2.assign_call_cnt);
/* The rebalance cb is not called on and empty revoke (unless partitions
* lost, which is not the case here) */
if (rebalance_cb1.revoke_call_cnt != 1)
Test::Fail(tostr() << "Expecting 1 revoke call on consumer 1, not: "
<< rebalance_cb1.revoke_call_cnt);
if (rebalance_cb2.revoke_call_cnt != 0)
Test::Fail(tostr() << "Expecting 0 revoke calls on consumer 2, not: "
<< rebalance_cb2.revoke_call_cnt);
}
/* Final state */
/* Expect both consumers to have 1 assigned partition (via net calculation in
* rebalance_cb) */
if (rebalance_cb1.partitions_assigned_net != 1)
Test::Fail(tostr()
<< "Expecting consumer 1 to have net 1 assigned partition, not: "
<< rebalance_cb1.partitions_assigned_net);
if (rebalance_cb2.partitions_assigned_net != 1)
Test::Fail(tostr()
<< "Expecting consumer 2 to have net 1 assigned partition, not: "
<< rebalance_cb2.partitions_assigned_net);
/* Expect both consumers to have 1 assigned partition (via ->assignment()
* query) */
expect_assignment(c1, 1);
expect_assignment(c2, 1);