-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
Copy pathrdkafka_sticky_assignor.c
4780 lines (3949 loc) · 204 KB
/
rdkafka_sticky_assignor.c
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 - The Apache Kafka C/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 "rdkafka_int.h"
#include "rdkafka_assignor.h"
#include "rdkafka_request.h"
#include "rdmap.h"
#include "rdunittest.h"
#include <stdarg.h>
#include <math.h> /* abs() */
/**
* @name KIP-54 and KIP-341 Sticky assignor.
*
* Closely mimicking the official Apache Kafka AbstractStickyAssignor
* implementation.
*/
/** FIXME
* Remaining:
* isSticky() -- used by tests
*/
/** @brief Assignor state from last rebalance */
typedef struct rd_kafka_sticky_assignor_state_s {
rd_kafka_topic_partition_list_t *prev_assignment;
int32_t generation_id;
} rd_kafka_sticky_assignor_state_t;
/**
* Auxilliary glue types
*/
/**
* @struct ConsumerPair_t represents a pair of consumer member ids involved in
* a partition reassignment, indicating a source consumer a partition
* is moving from and a destination partition the same partition is
* moving to.
*
* @sa PartitionMovements_t
*/
typedef struct ConsumerPair_s {
const char *src; /**< Source member id */
const char *dst; /**< Destination member id */
} ConsumerPair_t;
static ConsumerPair_t *ConsumerPair_new(const char *src, const char *dst) {
ConsumerPair_t *cpair;
cpair = rd_malloc(sizeof(*cpair));
cpair->src = src ? rd_strdup(src) : NULL;
cpair->dst = dst ? rd_strdup(dst) : NULL;
return cpair;
}
static void ConsumerPair_free(void *p) {
ConsumerPair_t *cpair = p;
if (cpair->src)
rd_free((void *)cpair->src);
if (cpair->dst)
rd_free((void *)cpair->dst);
rd_free(cpair);
}
static int ConsumerPair_cmp(const void *_a, const void *_b) {
const ConsumerPair_t *a = _a, *b = _b;
int r = strcmp(a->src ? a->src : "", b->src ? b->src : "");
if (r)
return r;
return strcmp(a->dst ? a->dst : "", b->dst ? b->dst : "");
}
static unsigned int ConsumerPair_hash(const void *_a) {
const ConsumerPair_t *a = _a;
return 31 * (a->src ? rd_map_str_hash(a->src) : 1) +
(a->dst ? rd_map_str_hash(a->dst) : 1);
}
typedef struct ConsumerGenerationPair_s {
const char *consumer; /**< Memory owned by caller */
int generation;
} ConsumerGenerationPair_t;
static void ConsumerGenerationPair_destroy(void *ptr) {
ConsumerGenerationPair_t *cgpair = ptr;
rd_free(cgpair);
}
/**
* @param consumer This memory will be referenced, not copied, and thus must
* outlive the ConsumerGenerationPair_t object.
*/
static ConsumerGenerationPair_t *
ConsumerGenerationPair_new(const char *consumer, int generation) {
ConsumerGenerationPair_t *cgpair = rd_malloc(sizeof(*cgpair));
cgpair->consumer = consumer;
cgpair->generation = generation;
return cgpair;
}
static int ConsumerGenerationPair_cmp_generation(const void *_a,
const void *_b) {
const ConsumerGenerationPair_t *a = _a, *b = _b;
return a->generation - b->generation;
}
/**
* Hash map types.
*
* Naming convention is:
* map_<keytype>_<valuetype>_t
*
* Where the keytype and valuetype are spoken names of the types and
* not the specific C types (since that'd be too long).
*/
typedef RD_MAP_TYPE(const char *,
rd_kafka_topic_partition_list_t *) map_str_toppar_list_t;
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
const char *) map_toppar_str_t;
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
rd_list_t *) map_toppar_list_t;
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
rd_kafka_metadata_partition_internal_t *) map_toppar_mdpi_t;
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
ConsumerGenerationPair_t *) map_toppar_cgpair_t;
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
ConsumerPair_t *) map_toppar_cpair_t;
typedef RD_MAP_TYPE(const ConsumerPair_t *,
rd_kafka_topic_partition_list_t *) map_cpair_toppar_list_t;
/* map<string, map<ConsumerPair*, topic_partition_list_t*>> */
typedef RD_MAP_TYPE(const char *,
map_cpair_toppar_list_t *) map_str_map_cpair_toppar_list_t;
typedef RD_MAP_TYPE(const char *, const char *) map_str_str_t;
/** Glue type helpers */
static map_cpair_toppar_list_t *map_cpair_toppar_list_t_new(void) {
map_cpair_toppar_list_t *map = rd_calloc(1, sizeof(*map));
RD_MAP_INIT(map, 0, ConsumerPair_cmp, ConsumerPair_hash, NULL,
rd_kafka_topic_partition_list_destroy_free);
return map;
}
static void map_cpair_toppar_list_t_free(void *ptr) {
map_cpair_toppar_list_t *map = ptr;
RD_MAP_DESTROY(map);
rd_free(map);
}
/** @struct Convenience struct for storing consumer/rack and toppar/rack
* mappings. */
typedef struct {
/** A map of member_id -> rack_id pairs. */
map_str_str_t member_id_to_rack_id;
/* A map of topic partition to rd_kafka_metadata_partition_internal_t */
map_toppar_mdpi_t toppar_to_mdpi;
} rd_kafka_rack_info_t;
/**
* @brief Initialize a rd_kafka_rack_info_t.
*
* @param topics
* @param topic_cnt
* @param mdi
*
* This struct is for convenience/easy grouping, and as a consequence, we avoid
* copying values. Thus, it is intended to be used within the lifetime of this
* function's arguments.
*
* @return rd_kafka_rack_info_t*
*/
static rd_kafka_rack_info_t *
rd_kafka_rack_info_new(rd_kafka_assignor_topic_t **topics,
size_t topic_cnt,
const rd_kafka_metadata_internal_t *mdi) {
int i;
size_t t;
rd_kafka_group_member_t *rkgm;
rd_kafka_rack_info_t *rkri = rd_calloc(1, sizeof(rd_kafka_rack_info_t));
if (!rd_kafka_use_rack_aware_assignment(topics, topic_cnt, mdi)) {
/* Free everything immediately, we aren't using rack aware
assignment, this struct is not applicable. */
rd_free(rkri);
return NULL;
}
rkri->member_id_to_rack_id = (map_str_str_t)RD_MAP_INITIALIZER(
0, rd_map_str_cmp, rd_map_str_hash,
NULL /* refs members.rkgm_member_id */,
NULL /* refs members.rkgm_rack_id */);
rkri->toppar_to_mdpi = (map_toppar_mdpi_t)RD_MAP_INITIALIZER(
0, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, NULL);
for (t = 0; t < topic_cnt; t++) {
RD_LIST_FOREACH(rkgm, &topics[t]->members, i) {
RD_MAP_SET(&rkri->member_id_to_rack_id,
rkgm->rkgm_member_id->str,
rkgm->rkgm_rack_id->str);
}
for (i = 0; i < topics[t]->metadata->partition_cnt; i++) {
rd_kafka_topic_partition_t *rkpart =
rd_kafka_topic_partition_new(
topics[t]->metadata->topic, i);
RD_MAP_SET(
&rkri->toppar_to_mdpi, rkpart,
&topics[t]->metadata_internal->partitions[i]);
}
}
return rkri;
}
/* Destroy a rd_kafka_rack_info_t. */
static void rd_kafka_rack_info_destroy(rd_kafka_rack_info_t *rkri) {
if (!rkri)
return;
RD_MAP_DESTROY(&rkri->member_id_to_rack_id);
RD_MAP_DESTROY(&rkri->toppar_to_mdpi);
rd_free(rkri);
}
/* Convenience function to bsearch inside the racks of a
* rd_kafka_metadata_partition_internal_t. */
static char *rd_kafka_partition_internal_find_rack(
rd_kafka_metadata_partition_internal_t *mdpi,
const char *rack) {
char **partition_racks = mdpi->racks;
size_t cnt = mdpi->racks_cnt;
void *res =
bsearch(&rack, partition_racks, cnt, sizeof(char *), rd_strcmp3);
if (res)
return *(char **)res;
return NULL;
}
/* Computes whether there is a rack mismatch between the rack of the consumer
* and the topic partition/any of its replicas. */
static rd_bool_t
rd_kafka_racks_mismatch(rd_kafka_rack_info_t *rkri,
const char *consumer,
const rd_kafka_topic_partition_t *topic_partition) {
const char *consumer_rack;
rd_kafka_metadata_partition_internal_t *mdpi;
if (rkri == NULL) /* Not using rack aware assignment */
return rd_false;
consumer_rack = RD_MAP_GET(&rkri->member_id_to_rack_id, consumer);
mdpi = RD_MAP_GET(&rkri->toppar_to_mdpi, topic_partition);
return consumer_rack != NULL &&
(mdpi == NULL ||
!rd_kafka_partition_internal_find_rack(mdpi, consumer_rack));
}
/**
* @struct Provides current state of partition movements between consumers
* for each topic, and possible movements for each partition.
*/
typedef struct PartitionMovements_s {
map_toppar_cpair_t partitionMovements;
map_str_map_cpair_toppar_list_t partitionMovementsByTopic;
} PartitionMovements_t;
static void PartitionMovements_init(PartitionMovements_t *pmov,
size_t topic_cnt) {
RD_MAP_INIT(&pmov->partitionMovements, topic_cnt * 3,
rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
NULL, ConsumerPair_free);
RD_MAP_INIT(&pmov->partitionMovementsByTopic, topic_cnt, rd_map_str_cmp,
rd_map_str_hash, NULL, map_cpair_toppar_list_t_free);
}
static void PartitionMovements_destroy(PartitionMovements_t *pmov) {
RD_MAP_DESTROY(&pmov->partitionMovementsByTopic);
RD_MAP_DESTROY(&pmov->partitionMovements);
}
static ConsumerPair_t *PartitionMovements_removeMovementRecordOfPartition(
PartitionMovements_t *pmov,
const rd_kafka_topic_partition_t *toppar) {
ConsumerPair_t *cpair;
map_cpair_toppar_list_t *partitionMovementsForThisTopic;
rd_kafka_topic_partition_list_t *plist;
cpair = RD_MAP_GET(&pmov->partitionMovements, toppar);
rd_assert(cpair);
partitionMovementsForThisTopic =
RD_MAP_GET(&pmov->partitionMovementsByTopic, toppar->topic);
plist = RD_MAP_GET(partitionMovementsForThisTopic, cpair);
rd_assert(plist);
rd_kafka_topic_partition_list_del(plist, toppar->topic,
toppar->partition);
if (plist->cnt == 0)
RD_MAP_DELETE(partitionMovementsForThisTopic, cpair);
if (RD_MAP_IS_EMPTY(partitionMovementsForThisTopic))
RD_MAP_DELETE(&pmov->partitionMovementsByTopic, toppar->topic);
return cpair;
}
static void PartitionMovements_addPartitionMovementRecord(
PartitionMovements_t *pmov,
const rd_kafka_topic_partition_t *toppar,
ConsumerPair_t *cpair) {
map_cpair_toppar_list_t *partitionMovementsForThisTopic;
rd_kafka_topic_partition_list_t *plist;
RD_MAP_SET(&pmov->partitionMovements, toppar, cpair);
partitionMovementsForThisTopic =
RD_MAP_GET_OR_SET(&pmov->partitionMovementsByTopic, toppar->topic,
map_cpair_toppar_list_t_new());
plist = RD_MAP_GET_OR_SET(partitionMovementsForThisTopic, cpair,
rd_kafka_topic_partition_list_new(16));
rd_kafka_topic_partition_list_add(plist, toppar->topic,
toppar->partition);
}
static void
PartitionMovements_movePartition(PartitionMovements_t *pmov,
const rd_kafka_topic_partition_t *toppar,
const char *old_consumer,
const char *new_consumer) {
if (RD_MAP_GET(&pmov->partitionMovements, toppar)) {
/* This partition has previously moved */
ConsumerPair_t *existing_cpair;
existing_cpair =
PartitionMovements_removeMovementRecordOfPartition(pmov,
toppar);
rd_assert(!rd_strcmp(existing_cpair->dst, old_consumer));
if (rd_strcmp(existing_cpair->src, new_consumer)) {
/* Partition is not moving back to its
* previous consumer */
PartitionMovements_addPartitionMovementRecord(
pmov, toppar,
ConsumerPair_new(existing_cpair->src,
new_consumer));
}
} else {
PartitionMovements_addPartitionMovementRecord(
pmov, toppar, ConsumerPair_new(old_consumer, new_consumer));
}
}
static const rd_kafka_topic_partition_t *
PartitionMovements_getTheActualPartitionToBeMoved(
PartitionMovements_t *pmov,
const rd_kafka_topic_partition_t *toppar,
const char *oldConsumer,
const char *newConsumer) {
ConsumerPair_t *cpair;
ConsumerPair_t reverse_cpair = {.src = newConsumer, .dst = oldConsumer};
map_cpair_toppar_list_t *partitionMovementsForThisTopic;
rd_kafka_topic_partition_list_t *plist;
if (!RD_MAP_GET(&pmov->partitionMovementsByTopic, toppar->topic))
return toppar;
cpair = RD_MAP_GET(&pmov->partitionMovements, toppar);
if (cpair) {
/* This partition has previously moved */
rd_assert(!rd_strcmp(oldConsumer, cpair->dst));
oldConsumer = cpair->src;
}
partitionMovementsForThisTopic =
RD_MAP_GET(&pmov->partitionMovementsByTopic, toppar->topic);
plist = RD_MAP_GET(partitionMovementsForThisTopic, &reverse_cpair);
if (!plist)
return toppar;
return &plist->elems[0];
}
#if FIXME
static rd_bool_t hasCycles(map_cpair_toppar_list_t *pairs) {
return rd_true; // FIXME
}
/**
* @remark This method is only used by the AbstractStickyAssignorTest
* in the Java client.
*/
static rd_bool_t PartitionMovements_isSticky(rd_kafka_t *rk,
PartitionMovements_t *pmov) {
const char *topic;
map_cpair_toppar_list_t *topicMovementPairs;
RD_MAP_FOREACH(topic, topicMovementPairs,
&pmov->partitionMovementsByTopic) {
if (hasCycles(topicMovementPairs)) {
const ConsumerPair_t *cpair;
const rd_kafka_topic_partition_list_t *partitions;
rd_kafka_log(
rk, LOG_ERR, "STICKY",
"Sticky assignor: Stickiness is violated for "
"topic %s: partition movements for this topic "
"occurred among the following consumers: ",
topic);
RD_MAP_FOREACH(cpair, partitions, topicMovementPairs) {
rd_kafka_log(rk, LOG_ERR, "STICKY", " %s -> %s",
cpair->src, cpair->dst);
}
if (partitions)
; /* Avoid unused warning */
return rd_false;
}
}
return rd_true;
}
#endif
/**
* @brief Comparator to sort ascendingly by rd_map_elem_t object value as
* topic partition list count, or by member id if the list count is
* identical.
* Used to sort sortedCurrentSubscriptions list.
*
* elem.key is the consumer member id string,
* elem.value is the partition list.
*/
static int sort_by_map_elem_val_toppar_list_cnt(const void *_a,
const void *_b) {
const rd_map_elem_t *a = _a, *b = _b;
const rd_kafka_topic_partition_list_t *al = a->value, *bl = b->value;
int r = al->cnt - bl->cnt;
if (r)
return r;
return strcmp((const char *)a->key, (const char *)b->key);
}
/**
* @brief Assign partition to the most eligible consumer.
*
* The assignment should improve the overall balance of the partition
* assignments to consumers.
* @returns true if partition was assigned, false otherwise.
*/
static rd_bool_t
maybeAssignPartition(const rd_kafka_topic_partition_t *partition,
rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_str_toppar_list_t *currentAssignment,
map_str_toppar_list_t *consumer2AllPotentialPartitions,
map_toppar_str_t *currentPartitionConsumer,
rd_kafka_rack_info_t *rkri) {
const rd_map_elem_t *elem;
int i;
RD_LIST_FOREACH(elem, sortedCurrentSubscriptions, i) {
const char *consumer = (const char *)elem->key;
const rd_kafka_topic_partition_list_t *partitions;
partitions =
RD_MAP_GET(consumer2AllPotentialPartitions, consumer);
if (!rd_kafka_topic_partition_list_find(
partitions, partition->topic, partition->partition))
continue;
if (rkri != NULL &&
rd_kafka_racks_mismatch(rkri, consumer, partition))
continue;
rd_kafka_topic_partition_list_add(
RD_MAP_GET(currentAssignment, consumer), partition->topic,
partition->partition);
RD_MAP_SET(currentPartitionConsumer,
rd_kafka_topic_partition_copy(partition), consumer);
/* Re-sort sortedCurrentSubscriptions since this consumer's
* assignment count has increased.
* This is an O(N) operation since it is a single shuffle. */
rd_list_sort(sortedCurrentSubscriptions,
sort_by_map_elem_val_toppar_list_cnt);
return rd_true;
}
return rd_false;
}
/**
* @returns true if the partition has two or more potential consumers.
*/
static RD_INLINE rd_bool_t partitionCanParticipateInReassignment(
const rd_kafka_topic_partition_t *partition,
map_toppar_list_t *partition2AllPotentialConsumers) {
rd_list_t *consumers;
if (!(consumers =
RD_MAP_GET(partition2AllPotentialConsumers, partition)))
return rd_false;
return rd_list_cnt(consumers) >= 2;
}
/**
* @returns true if consumer can participate in reassignment based on
* its current assignment.
*/
static RD_INLINE rd_bool_t consumerCanParticipateInReassignment(
rd_kafka_t *rk,
const char *consumer,
map_str_toppar_list_t *currentAssignment,
map_str_toppar_list_t *consumer2AllPotentialPartitions,
map_toppar_list_t *partition2AllPotentialConsumers) {
const rd_kafka_topic_partition_list_t *currentPartitions =
RD_MAP_GET(currentAssignment, consumer);
int currentAssignmentSize = currentPartitions->cnt;
int maxAssignmentSize =
RD_MAP_GET(consumer2AllPotentialPartitions, consumer)->cnt;
int i;
/* FIXME: And then what? Is this a local error? If so, assert. */
if (currentAssignmentSize > maxAssignmentSize)
rd_kafka_log(rk, LOG_ERR, "STICKY",
"Sticky assignor error: "
"Consumer %s is assigned more partitions (%d) "
"than the maximum possible (%d)",
consumer, currentAssignmentSize,
maxAssignmentSize);
/* If a consumer is not assigned all its potential partitions it is
* subject to reassignment. */
if (currentAssignmentSize < maxAssignmentSize)
return rd_true;
/* If any of the partitions assigned to a consumer is subject to
* reassignment the consumer itself is subject to reassignment. */
for (i = 0; i < currentPartitions->cnt; i++) {
const rd_kafka_topic_partition_t *partition =
¤tPartitions->elems[i];
if (partitionCanParticipateInReassignment(
partition, partition2AllPotentialConsumers))
return rd_true;
}
return rd_false;
}
/**
* @brief Process moving partition from old consumer to new consumer.
*/
static void processPartitionMovement(
rd_kafka_t *rk,
PartitionMovements_t *partitionMovements,
const rd_kafka_topic_partition_t *partition,
const char *newConsumer,
map_str_toppar_list_t *currentAssignment,
rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_toppar_str_t *currentPartitionConsumer) {
const char *oldConsumer =
RD_MAP_GET(currentPartitionConsumer, partition);
PartitionMovements_movePartition(partitionMovements, partition,
oldConsumer, newConsumer);
rd_kafka_topic_partition_list_add(
RD_MAP_GET(currentAssignment, newConsumer), partition->topic,
partition->partition);
rd_kafka_topic_partition_list_del(
RD_MAP_GET(currentAssignment, oldConsumer), partition->topic,
partition->partition);
RD_MAP_SET(currentPartitionConsumer,
rd_kafka_topic_partition_copy(partition), newConsumer);
/* Re-sort after assignment count has changed. */
rd_list_sort(sortedCurrentSubscriptions,
sort_by_map_elem_val_toppar_list_cnt);
rd_kafka_dbg(rk, ASSIGNOR, "STICKY",
"%s [%" PRId32 "] %sassigned to %s (from %s)",
partition->topic, partition->partition,
oldConsumer ? "re" : "", newConsumer,
oldConsumer ? oldConsumer : "(none)");
}
/**
* @brief Reassign \p partition to \p newConsumer
*/
static void reassignPartitionToConsumer(
rd_kafka_t *rk,
PartitionMovements_t *partitionMovements,
const rd_kafka_topic_partition_t *partition,
map_str_toppar_list_t *currentAssignment,
rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_toppar_str_t *currentPartitionConsumer,
const char *newConsumer) {
const char *consumer = RD_MAP_GET(currentPartitionConsumer, partition);
const rd_kafka_topic_partition_t *partitionToBeMoved;
/* Find the correct partition movement considering
* the stickiness requirement. */
partitionToBeMoved = PartitionMovements_getTheActualPartitionToBeMoved(
partitionMovements, partition, consumer, newConsumer);
processPartitionMovement(rk, partitionMovements, partitionToBeMoved,
newConsumer, currentAssignment,
sortedCurrentSubscriptions,
currentPartitionConsumer);
}
/**
* @brief Reassign \p partition to an eligible new consumer.
*/
static void
reassignPartition(rd_kafka_t *rk,
PartitionMovements_t *partitionMovements,
const rd_kafka_topic_partition_t *partition,
map_str_toppar_list_t *currentAssignment,
rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_toppar_str_t *currentPartitionConsumer,
map_str_toppar_list_t *consumer2AllPotentialPartitions) {
const rd_map_elem_t *elem;
int i;
/* Find the new consumer */
RD_LIST_FOREACH(elem, sortedCurrentSubscriptions, i) {
const char *newConsumer = (const char *)elem->key;
if (rd_kafka_topic_partition_list_find(
RD_MAP_GET(consumer2AllPotentialPartitions,
newConsumer),
partition->topic, partition->partition)) {
reassignPartitionToConsumer(
rk, partitionMovements, partition,
currentAssignment, sortedCurrentSubscriptions,
currentPartitionConsumer, newConsumer);
return;
}
}
rd_assert(!*"reassignPartition(): no new consumer found");
}
/**
* @brief Determine if the current assignment is balanced.
*
* @param currentAssignment the assignment whose balance needs to be checked
* @param sortedCurrentSubscriptions an ascending sorted set of consumers based
* on how many topic partitions are already
* assigned to them
* @param consumer2AllPotentialPartitions a mapping of all consumers to all
* potential topic partitions that can be
* assigned to them.
* This parameter is called
* allSubscriptions in the Java
* implementation, but we choose this
* name to be more consistent with its
* use elsewhere in the code.
* @param partition2AllPotentialConsumers a mapping of all partitions to
* all potential consumers.
*
* @returns true if the given assignment is balanced; false otherwise
*/
static rd_bool_t
isBalanced(rd_kafka_t *rk,
map_str_toppar_list_t *currentAssignment,
const rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_str_toppar_list_t *consumer2AllPotentialPartitions,
map_toppar_list_t *partition2AllPotentialConsumers) {
int minimum = ((const rd_kafka_topic_partition_list_t
*)((const rd_map_elem_t *)rd_list_first(
sortedCurrentSubscriptions))
->value)
->cnt;
int maximum = ((const rd_kafka_topic_partition_list_t
*)((const rd_map_elem_t *)rd_list_last(
sortedCurrentSubscriptions))
->value)
->cnt;
/* Iterators */
const rd_kafka_topic_partition_list_t *partitions;
const char *consumer;
const rd_map_elem_t *elem;
int i;
/* The assignment is balanced if minimum and maximum numbers of
* partitions assigned to consumers differ by at most one. */
if (minimum >= maximum - 1) {
rd_kafka_dbg(rk, ASSIGNOR, "STICKY",
"Assignment is balanced: "
"minimum %d and maximum %d partitions assigned "
"to each consumer",
minimum, maximum);
return rd_true;
}
/* Mapping from partitions to the consumer assigned to them */
map_toppar_str_t allPartitions = RD_MAP_INITIALIZER(
RD_MAP_CNT(partition2AllPotentialConsumers),
rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
NULL /* references currentAssignment */,
NULL /* references currentAssignment */);
/* Create a mapping from partitions to the consumer assigned to them */
RD_MAP_FOREACH(consumer, partitions, currentAssignment) {
for (i = 0; i < partitions->cnt; i++) {
const rd_kafka_topic_partition_t *partition =
&partitions->elems[i];
const char *existing;
if ((existing = RD_MAP_GET(&allPartitions, partition)))
rd_kafka_log(rk, LOG_ERR, "STICKY",
"Sticky assignor: %s [%" PRId32
"] "
"is assigned to more than one "
"consumer (%s and %s)",
partition->topic,
partition->partition, existing,
consumer);
RD_MAP_SET(&allPartitions, partition, consumer);
}
}
/* For each consumer that does not have all the topic partitions it
* can get make sure none of the topic partitions it could but did
* not get cannot be moved to it, because that would break the balance.
*
* Note: Since sortedCurrentSubscriptions elements are pointers to
* currentAssignment's element we get both the consumer
* and partition list in elem here. */
RD_LIST_FOREACH(elem, sortedCurrentSubscriptions, i) {
const char *consumer = (const char *)elem->key;
const rd_kafka_topic_partition_list_t *potentialTopicPartitions;
const rd_kafka_topic_partition_list_t *consumerPartitions;
consumerPartitions =
(const rd_kafka_topic_partition_list_t *)elem->value;
potentialTopicPartitions =
RD_MAP_GET(consumer2AllPotentialPartitions, consumer);
/* Skip if this consumer already has all the topic partitions
* it can get. */
if (consumerPartitions->cnt == potentialTopicPartitions->cnt)
continue;
/* Otherwise make sure it can't get any more partitions */
for (i = 0; i < potentialTopicPartitions->cnt; i++) {
const rd_kafka_topic_partition_t *partition =
&potentialTopicPartitions->elems[i];
const char *otherConsumer;
int otherConsumerPartitionCount;
if (rd_kafka_topic_partition_list_find(
consumerPartitions, partition->topic,
partition->partition))
continue;
otherConsumer = RD_MAP_GET(&allPartitions, partition);
otherConsumerPartitionCount =
RD_MAP_GET(currentAssignment, otherConsumer)->cnt;
if (consumerPartitions->cnt <
otherConsumerPartitionCount) {
rd_kafka_dbg(
rk, ASSIGNOR, "STICKY",
"%s [%" PRId32
"] can be moved from "
"consumer %s (%d partition(s)) to "
"consumer %s (%d partition(s)) "
"for a more balanced assignment",
partition->topic, partition->partition,
otherConsumer, otherConsumerPartitionCount,
consumer, consumerPartitions->cnt);
RD_MAP_DESTROY(&allPartitions);
return rd_false;
}
}
}
RD_MAP_DESTROY(&allPartitions);
return rd_true;
}
/**
* @brief Perform reassignment.
*
* @returns true if reassignment was performed.
*/
static rd_bool_t
performReassignments(rd_kafka_t *rk,
PartitionMovements_t *partitionMovements,
rd_kafka_topic_partition_list_t *reassignablePartitions,
map_str_toppar_list_t *currentAssignment,
map_toppar_cgpair_t *prevAssignment,
rd_list_t *sortedCurrentSubscriptions /*rd_map_elem_t*/,
map_str_toppar_list_t *consumer2AllPotentialPartitions,
map_toppar_list_t *partition2AllPotentialConsumers,
map_toppar_str_t *currentPartitionConsumer,
rd_kafka_rack_info_t *rkri) {
rd_bool_t reassignmentPerformed = rd_false;
rd_bool_t modified, saveIsBalanced = rd_false;
int iterations = 0;
/* Repeat reassignment until no partition can be moved to
* improve the balance. */
do {
int i;
iterations++;
modified = rd_false;
/* Reassign all reassignable partitions (starting from the
* partition with least potential consumers and if needed)
* until the full list is processed or a balance is achieved. */
for (i = 0; i < reassignablePartitions->cnt &&
!isBalanced(rk, currentAssignment,
sortedCurrentSubscriptions,
consumer2AllPotentialPartitions,
partition2AllPotentialConsumers);
i++) {
const rd_kafka_topic_partition_t *partition =
&reassignablePartitions->elems[i];
const rd_list_t *consumers = RD_MAP_GET(
partition2AllPotentialConsumers, partition);
const char *consumer, *otherConsumer;
const ConsumerGenerationPair_t *prevcgp;
const rd_kafka_topic_partition_list_t *currAssignment;
int j;
rd_bool_t found_rack;
const char *consumer_rack = NULL;
rd_kafka_metadata_partition_internal_t *mdpi = NULL;
/* FIXME: Is this a local error/bug? If so, assert */
if (rd_list_cnt(consumers) <= 1)
rd_kafka_log(
rk, LOG_ERR, "STICKY",
"Sticky assignor: expected more than "
"one potential consumer for partition "
"%s [%" PRId32 "]",
partition->topic, partition->partition);
/* The partition must have a current consumer */
consumer =
RD_MAP_GET(currentPartitionConsumer, partition);
rd_assert(consumer);
currAssignment =
RD_MAP_GET(currentAssignment, consumer);
prevcgp = RD_MAP_GET(prevAssignment, partition);
if (prevcgp &&
currAssignment->cnt >
RD_MAP_GET(currentAssignment, prevcgp->consumer)
->cnt +
1) {
reassignPartitionToConsumer(
rk, partitionMovements, partition,
currentAssignment,
sortedCurrentSubscriptions,
currentPartitionConsumer,
prevcgp->consumer);
reassignmentPerformed = rd_true;
modified = rd_true;
continue;
}
/* Check if a better-suited consumer exists for the
* partition; if so, reassign it. Use consumer within
* rack if possible. */
if (rkri) {
consumer_rack = RD_MAP_GET(
&rkri->member_id_to_rack_id, consumer);
mdpi = RD_MAP_GET(&rkri->toppar_to_mdpi,
partition);
}
found_rack = rd_false;
if (consumer_rack != NULL && mdpi != NULL &&
mdpi->racks_cnt > 0 &&
rd_kafka_partition_internal_find_rack(
mdpi, consumer_rack)) {
RD_LIST_FOREACH(otherConsumer, consumers, j) {
/* No need for rkri == NULL check, that
* is guaranteed if we're inside this if
* block. */
const char *other_consumer_rack =
RD_MAP_GET(
&rkri->member_id_to_rack_id,
otherConsumer);
if (other_consumer_rack == NULL ||
!rd_kafka_partition_internal_find_rack(
mdpi, other_consumer_rack))
continue;
if (currAssignment->cnt <=
RD_MAP_GET(currentAssignment,
otherConsumer)
->cnt +
1)
continue;
reassignPartition(
rk, partitionMovements, partition,
currentAssignment,