-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
Copy pathrdkafka_cgrp.c
6116 lines (5014 loc) · 236 KB
/
rdkafka_cgrp.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 - Apache Kafka C library
*
* Copyright (c) 2012-2015, Magnus Edenhill
* 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_broker.h"
#include "rdkafka_request.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_assignor.h"
#include "rdkafka_offset.h"
#include "rdkafka_metadata.h"
#include "rdkafka_cgrp.h"
#include "rdkafka_interceptor.h"
#include "rdmap.h"
#include "rdunittest.h"
#include <ctype.h>
#include <stdarg.h>
static void rd_kafka_cgrp_offset_commit_tmr_cb (rd_kafka_timers_t *rkts,
void *arg);
static rd_kafka_error_t *
rd_kafka_cgrp_assign (rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment);
static rd_kafka_error_t *rd_kafka_cgrp_unassign (rd_kafka_cgrp_t *rkcg);
static rd_kafka_error_t *
rd_kafka_cgrp_incremental_assign (rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t
*partitions);
static rd_kafka_error_t *
rd_kafka_cgrp_incremental_unassign (rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t
*partitions);
static rd_kafka_op_res_t
rd_kafka_cgrp_op_serve (rd_kafka_t *rk, rd_kafka_q_t *rkq,
rd_kafka_op_t *rko, rd_kafka_q_cb_type_t cb_type,
void *opaque);
static void rd_kafka_cgrp_group_leader_reset (rd_kafka_cgrp_t *rkcg,
const char *reason);
static RD_INLINE int rd_kafka_cgrp_try_terminate (rd_kafka_cgrp_t *rkcg);
static void rd_kafka_cgrp_revoke_all_rejoin (rd_kafka_cgrp_t *rkcg,
rd_bool_t assignment_lost,
rd_bool_t initiating,
const char *reason);
static void rd_kafka_cgrp_revoke_all_rejoin_maybe (rd_kafka_cgrp_t *rkcg,
rd_bool_t
assignment_lost,
rd_bool_t initiating,
const char *reason);
static void rd_kafka_cgrp_group_is_rebalancing (rd_kafka_cgrp_t *rkcg);
static void
rd_kafka_cgrp_max_poll_interval_check_tmr_cb (rd_kafka_timers_t *rkts,
void *arg);
static rd_kafka_resp_err_t
rd_kafka_cgrp_subscribe (rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist);
static void rd_kafka_cgrp_group_assignment_set (
rd_kafka_cgrp_t *rkcg,
const rd_kafka_topic_partition_list_t *partitions);
static void rd_kafka_cgrp_group_assignment_modify (
rd_kafka_cgrp_t *rkcg,
rd_bool_t add,
const rd_kafka_topic_partition_list_t *partitions);
static void
rd_kafka_cgrp_handle_assignment (rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment);
/**
* @returns true if the current assignment is lost.
*/
rd_bool_t rd_kafka_cgrp_assignment_is_lost (rd_kafka_cgrp_t *rkcg) {
return rd_atomic32_get(&rkcg->rkcg_assignment_lost) != 0;
}
/**
* @brief Call when the current assignment has been lost, with a
* human-readable reason.
*/
static void rd_kafka_cgrp_assignment_set_lost (rd_kafka_cgrp_t *rkcg,
char *fmt, ...)
RD_FORMAT(printf, 2, 3);
static void rd_kafka_cgrp_assignment_set_lost (rd_kafka_cgrp_t *rkcg,
char *fmt, ...) {
va_list ap;
char reason[256];
if (!rkcg->rkcg_group_assignment)
return;
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
va_end(ap);
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER|RD_KAFKA_DBG_CGRP, "LOST",
"Group \"%s\": "
"current assignment of %d partition(s) lost: %s",
rkcg->rkcg_group_id->str,
rkcg->rkcg_group_assignment->cnt,
reason);
rd_atomic32_set(&rkcg->rkcg_assignment_lost, rd_true);
}
/**
* @brief Call when the current assignment is no longer considered lost, with a
* human-readable reason.
*/
static void rd_kafka_cgrp_assignment_clear_lost (rd_kafka_cgrp_t *rkcg,
char *fmt, ...) {
va_list ap;
char reason[256];
if (!rd_atomic32_get(&rkcg->rkcg_assignment_lost))
return;
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
va_end(ap);
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER|RD_KAFKA_DBG_CGRP, "LOST",
"Group \"%s\": "
"current assignment no longer considered lost: %s",
rkcg->rkcg_group_id->str, reason);
rd_atomic32_set(&rkcg->rkcg_assignment_lost, rd_false);
}
/**
* @brief The rebalance protocol currently in use. This will be
* RD_KAFKA_REBALANCE_PROTOCOL_NONE if the consumer has not
* (yet) joined a group, else it will match the rebalance
* protocol of the configured assignor(s).
*
* @locality main thread
*/
rd_kafka_rebalance_protocol_t
rd_kafka_cgrp_rebalance_protocol (rd_kafka_cgrp_t *rkcg) {
if (!rkcg->rkcg_assignor)
return RD_KAFKA_REBALANCE_PROTOCOL_NONE;
return rkcg->rkcg_assignor->rkas_protocol;
}
/**
* @returns true if the cgrp is awaiting a protocol response. This prohibits
* the join-state machine to proceed before the current state
* is done.
*/
static rd_bool_t rd_kafka_cgrp_awaiting_response (rd_kafka_cgrp_t *rkcg) {
return rkcg->rkcg_wait_resp != -1;
}
/**
* @brief Set flag indicating we are waiting for a coordinator response
* for the given request.
*
* This is used for specific requests to postpone rejoining the group if
* there are outstanding JoinGroup or SyncGroup requests.
*
* @locality main thread
*/
static void rd_kafka_cgrp_set_wait_resp (rd_kafka_cgrp_t *rkcg,
int16_t ApiKey) {
rd_assert(rkcg->rkcg_wait_resp == -1);
rkcg->rkcg_wait_resp = ApiKey;
}
/**
* @brief Clear the flag that says we're waiting for a coordinator response
* for the given \p request.
*
* @param request Original request, possibly NULL (for errors).
*
* @locality main thread
*/
static void rd_kafka_cgrp_clear_wait_resp (rd_kafka_cgrp_t *rkcg,
int16_t ApiKey) {
rd_assert(rkcg->rkcg_wait_resp == ApiKey);
rkcg->rkcg_wait_resp = -1;
}
/**
* @struct Auxillary glue type used for COOPERATIVE rebalance set operations.
*/
typedef struct PartitionMemberInfo_s {
const rd_kafka_group_member_t *member;
rd_bool_t members_match;
} PartitionMemberInfo_t;
static PartitionMemberInfo_t *PartitionMemberInfo_new (
const rd_kafka_group_member_t *member,
rd_bool_t members_match) {
PartitionMemberInfo_t *pmi;
pmi = rd_calloc(1, sizeof(*pmi));
pmi->member = member;
pmi->members_match = members_match;
return pmi;
}
static void PartitionMemberInfo_free (void *p) {
PartitionMemberInfo_t *pmi = p;
rd_free(pmi);
}
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
PartitionMemberInfo_t *) map_toppar_member_info_t;
/**
* @returns true if consumer has joined the group and thus requires a leave.
*/
#define RD_KAFKA_CGRP_HAS_JOINED(rkcg) \
(rkcg->rkcg_member_id != NULL && \
RD_KAFKAP_STR_LEN((rkcg)->rkcg_member_id) > 0)
/**
* @returns true if cgrp is waiting for a rebalance_cb to be handled by
* the application.
*/
#define RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg) \
((rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL)
/**
* @returns true if a rebalance is in progress.
*
* 1. In WAIT_JOIN or WAIT_METADATA state with a member-id set,
* this happens on rejoin.
* 2. In WAIT_SYNC waiting for the group to rebalance on the broker.
* 3. in *_WAIT_UNASSIGN_TO_COMPLETE waiting for unassigned partitions to
* stop fetching, et.al.
* 4. In _WAIT_*ASSIGN_CALL waiting for the application to handle the
* assignment changes in its rebalance callback and then call *assign().
* 5. An incremental rebalancing is in progress.
* 6. A rebalance-induced rejoin is in progress.
*/
#define RD_KAFKA_CGRP_REBALANCING(rkcg) \
((RD_KAFKA_CGRP_HAS_JOINED(rkcg) && \
((rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA)) || \
(rkcg)->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_INCR_UNASSIGN_TO_COMPLETE || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL || \
(rkcg)->rkcg_rebalance_incr_assignment != NULL || \
(rkcg)->rkcg_rebalance_rejoin)
const char *rd_kafka_cgrp_state_names[] = {
"init",
"term",
"query-coord",
"wait-coord",
"wait-broker",
"wait-broker-transport",
"up"
};
const char *rd_kafka_cgrp_join_state_names[] = {
"init",
"wait-join",
"wait-metadata",
"wait-sync",
"wait-assign-call",
"wait-unassign-call",
"wait-unassign-to-complete",
"wait-incr-unassign-to-complete",
"steady",
};
/**
* @brief Change the cgrp state.
*
* @returns 1 if the state was changed, else 0.
*/
static int rd_kafka_cgrp_set_state (rd_kafka_cgrp_t *rkcg, int state) {
if ((int)rkcg->rkcg_state == state)
return 0;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPSTATE",
"Group \"%.*s\" changed state %s -> %s "
"(join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_state_names[state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
rkcg->rkcg_state = state;
rkcg->rkcg_ts_statechange = rd_clock();
rd_kafka_brokers_broadcast_state_change(rkcg->rkcg_rk);
return 1;
}
void rd_kafka_cgrp_set_join_state (rd_kafka_cgrp_t *rkcg, int join_state) {
if ((int)rkcg->rkcg_join_state == join_state)
return;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPJOINSTATE",
"Group \"%.*s\" changed join state %s -> %s "
"(state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rd_kafka_cgrp_join_state_names[join_state],
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
rkcg->rkcg_join_state = join_state;
}
void rd_kafka_cgrp_destroy_final (rd_kafka_cgrp_t *rkcg) {
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_subscription);
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_group_leader.members);
rd_kafka_cgrp_set_member_id(rkcg, NULL);
if (rkcg->rkcg_group_instance_id)
rd_kafkap_str_destroy(rkcg->rkcg_group_instance_id);
rd_kafka_q_destroy_owner(rkcg->rkcg_q);
rd_kafka_q_destroy_owner(rkcg->rkcg_ops);
rd_kafka_q_destroy_owner(rkcg->rkcg_wait_coord_q);
rd_kafka_assert(rkcg->rkcg_rk, TAILQ_EMPTY(&rkcg->rkcg_topics));
rd_kafka_assert(rkcg->rkcg_rk, rd_list_empty(&rkcg->rkcg_toppars));
rd_list_destroy(&rkcg->rkcg_toppars);
rd_list_destroy(rkcg->rkcg_subscribed_topics);
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_errored_topics);
if (rkcg->rkcg_assignor && rkcg->rkcg_assignor->rkas_destroy_state_cb)
rkcg->rkcg_assignor->rkas_destroy_state_cb(
rkcg->rkcg_assignor_state);
rd_free(rkcg);
}
/**
* @brief Update the absolute session timeout following a successfull
* response from the coordinator.
* This timeout is used to enforce the session timeout in the
* consumer itself.
*
* @param reset if true the timeout is updated even if the session has expired.
*/
static RD_INLINE void
rd_kafka_cgrp_update_session_timeout (rd_kafka_cgrp_t *rkcg, rd_bool_t reset) {
if (reset || rkcg->rkcg_ts_session_timeout != 0)
rkcg->rkcg_ts_session_timeout = rd_clock() +
(rkcg->rkcg_rk->rk_conf.group_session_timeout_ms*1000);
}
rd_kafka_cgrp_t *rd_kafka_cgrp_new (rd_kafka_t *rk,
const rd_kafkap_str_t *group_id,
const rd_kafkap_str_t *client_id) {
rd_kafka_cgrp_t *rkcg;
rkcg = rd_calloc(1, sizeof(*rkcg));
rkcg->rkcg_rk = rk;
rkcg->rkcg_group_id = group_id;
rkcg->rkcg_client_id = client_id;
rkcg->rkcg_coord_id = -1;
rkcg->rkcg_generation_id = -1;
rkcg->rkcg_wait_resp = -1;
rkcg->rkcg_ops = rd_kafka_q_new(rk);
rkcg->rkcg_ops->rkq_serve = rd_kafka_cgrp_op_serve;
rkcg->rkcg_ops->rkq_opaque = rkcg;
rkcg->rkcg_wait_coord_q = rd_kafka_q_new(rk);
rkcg->rkcg_wait_coord_q->rkq_serve = rkcg->rkcg_ops->rkq_serve;
rkcg->rkcg_wait_coord_q->rkq_opaque = rkcg->rkcg_ops->rkq_opaque;
rkcg->rkcg_q = rd_kafka_q_new(rk);
rkcg->rkcg_group_instance_id =
rd_kafkap_str_new(rk->rk_conf.group_instance_id, -1);
TAILQ_INIT(&rkcg->rkcg_topics);
rd_list_init(&rkcg->rkcg_toppars, 32, NULL);
rd_kafka_cgrp_set_member_id(rkcg, "");
rkcg->rkcg_subscribed_topics =
rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
rd_interval_init(&rkcg->rkcg_coord_query_intvl);
rd_interval_init(&rkcg->rkcg_heartbeat_intvl);
rd_interval_init(&rkcg->rkcg_join_intvl);
rd_interval_init(&rkcg->rkcg_timeout_scan_intvl);
rd_atomic32_init(&rkcg->rkcg_assignment_lost, rd_false);
rkcg->rkcg_errored_topics = rd_kafka_topic_partition_list_new(0);
/* Create a logical group coordinator broker to provide
* a dedicated connection for group coordination.
* This is needed since JoinGroup may block for up to
* max.poll.interval.ms, effectively blocking and timing out
* any other protocol requests (such as Metadata).
* The address for this broker will be updated when
* the group coordinator is assigned. */
rkcg->rkcg_coord = rd_kafka_broker_add_logical(rk, "GroupCoordinator");
if (rk->rk_conf.enable_auto_commit &&
rk->rk_conf.auto_commit_interval_ms > 0)
rd_kafka_timer_start(&rk->rk_timers,
&rkcg->rkcg_offset_commit_tmr,
rk->rk_conf.
auto_commit_interval_ms * 1000ll,
rd_kafka_cgrp_offset_commit_tmr_cb,
rkcg);
return rkcg;
}
/**
* @brief Set the group coordinator broker.
*/
static void rd_kafka_cgrp_coord_set_broker (rd_kafka_cgrp_t *rkcg,
rd_kafka_broker_t *rkb) {
rd_assert(rkcg->rkcg_curr_coord == NULL);
rd_assert(RD_KAFKA_CGRP_BROKER_IS_COORD(rkcg, rkb));
rkcg->rkcg_curr_coord = rkb;
rd_kafka_broker_keep(rkb);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORDSET",
"Group \"%.*s\" coordinator set to broker %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_broker_name(rkb));
/* Reset query interval to trigger an immediate
* coord query if required */
if (!rd_interval_disabled(&rkcg->rkcg_coord_query_intvl))
rd_interval_reset(&rkcg->rkcg_coord_query_intvl);
rd_kafka_cgrp_set_state(rkcg,
RD_KAFKA_CGRP_STATE_WAIT_BROKER_TRANSPORT);
rd_kafka_broker_persistent_connection_add(
rkcg->rkcg_coord, &rkcg->rkcg_coord->rkb_persistconn.coord);
/* Set the logical coordinator's nodename to the
* proper broker's nodename, this will trigger a (re)connect
* to the new address. */
rd_kafka_broker_set_nodename(rkcg->rkcg_coord, rkb);
}
/**
* @brief Reset/clear the group coordinator broker.
*/
static void rd_kafka_cgrp_coord_clear_broker (rd_kafka_cgrp_t *rkcg) {
rd_kafka_broker_t *rkb = rkcg->rkcg_curr_coord;
rd_assert(rkcg->rkcg_curr_coord);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORDCLEAR",
"Group \"%.*s\" broker %s is no longer coordinator",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_broker_name(rkb));
rd_assert(rkcg->rkcg_coord);
rd_kafka_broker_persistent_connection_del(
rkcg->rkcg_coord,
&rkcg->rkcg_coord->rkb_persistconn.coord);
/* Clear the ephemeral broker's nodename.
* This will also trigger a disconnect. */
rd_kafka_broker_set_nodename(rkcg->rkcg_coord, NULL);
rkcg->rkcg_curr_coord = NULL;
rd_kafka_broker_destroy(rkb); /* from set_coord_broker() */
}
/**
* @brief Update/set the group coordinator.
*
* Will do nothing if there's been no change.
*
* @returns 1 if the coordinator, or state, was updated, else 0.
*/
static int rd_kafka_cgrp_coord_update (rd_kafka_cgrp_t *rkcg,
int32_t coord_id) {
/* Don't do anything while terminating */
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_TERM)
return 0;
/* Check if coordinator changed */
if (rkcg->rkcg_coord_id != coord_id) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPCOORD",
"Group \"%.*s\" changing coordinator %"PRId32
" -> %"PRId32,
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_coord_id, coord_id);
/* Update coord id */
rkcg->rkcg_coord_id = coord_id;
/* Clear previous broker handle, if any */
if (rkcg->rkcg_curr_coord)
rd_kafka_cgrp_coord_clear_broker(rkcg);
}
if (rkcg->rkcg_curr_coord) {
/* There is already a known coordinator and a
* corresponding broker handle. */
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP)
return rd_kafka_cgrp_set_state(
rkcg,
RD_KAFKA_CGRP_STATE_WAIT_BROKER_TRANSPORT);
} else if (rkcg->rkcg_coord_id != -1) {
rd_kafka_broker_t *rkb;
/* Try to find the coordinator broker handle */
rd_kafka_rdlock(rkcg->rkcg_rk);
rkb = rd_kafka_broker_find_by_nodeid(rkcg->rkcg_rk, coord_id);
rd_kafka_rdunlock(rkcg->rkcg_rk);
/* It is possible, due to stale metadata, that the
* coordinator id points to a broker we still don't know
* about. In this case the client will continue
* querying metadata and querying for the coordinator
* until a match is found. */
if (rkb) {
/* Coordinator is known and broker handle exists */
rd_kafka_cgrp_coord_set_broker(rkcg, rkb);
rd_kafka_broker_destroy(rkb); /*from find_by_nodeid()*/
return 1;
} else {
/* Coordinator is known but no corresponding
* broker handle. */
return rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_WAIT_BROKER);
}
} else {
/* Coordinator still not known, re-query */
if (rkcg->rkcg_state >= RD_KAFKA_CGRP_STATE_WAIT_COORD)
return rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
}
return 0; /* no change */
}
/**
* Handle FindCoordinator response
*/
static void rd_kafka_cgrp_handle_FindCoordinator (rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
int32_t CoordId;
rd_kafkap_str_t CoordHost = RD_ZERO_INIT;
int32_t CoordPort;
rd_kafka_cgrp_t *rkcg = opaque;
struct rd_kafka_metadata_broker mdb = RD_ZERO_INIT;
char *errstr = NULL;
int actions;
if (likely(!(ErrorCode = err))) {
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 1) {
rd_kafkap_str_t ErrorMsg;
rd_kafka_buf_read_str(rkbuf, &ErrorMsg);
if (!RD_KAFKAP_STR_IS_NULL(&ErrorMsg))
RD_KAFKAP_STR_DUPA(&errstr, &ErrorMsg);
}
rd_kafka_buf_read_i32(rkbuf, &CoordId);
rd_kafka_buf_read_str(rkbuf, &CoordHost);
rd_kafka_buf_read_i32(rkbuf, &CoordPort);
}
if (ErrorCode)
goto err;
mdb.id = CoordId;
RD_KAFKAP_STR_DUPA(&mdb.host, &CoordHost);
mdb.port = CoordPort;
rd_rkb_dbg(rkb, CGRP, "CGRPCOORD",
"Group \"%.*s\" coordinator is %s:%i id %"PRId32,
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
mdb.host, mdb.port, mdb.id);
rd_kafka_broker_update(rkb->rkb_rk, rkb->rkb_proto, &mdb, NULL);
rd_kafka_cgrp_coord_update(rkcg, CoordId);
rd_kafka_cgrp_serve(rkcg); /* Serve updated state, if possible */
return;
err_parse: /* Parse error */
ErrorCode = rkbuf->rkbuf_err;
/* FALLTHRU */
err:
if (!errstr)
errstr = (char *)rd_kafka_err2str(ErrorCode);
rd_rkb_dbg(rkb, CGRP, "CGRPCOORD",
"Group \"%.*s\" FindCoordinator response error: %s: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_err2name(ErrorCode), errstr);
if (ErrorCode == RD_KAFKA_RESP_ERR__DESTROY)
return;
actions = rd_kafka_err_action(
rkb, ErrorCode, request,
RD_KAFKA_ERR_ACTION_RETRY|RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE,
RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR__TRANSPORT,
RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR__TIMED_OUT,
RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE,
RD_KAFKA_ERR_ACTION_END);
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
rd_kafka_cgrp_coord_update(rkcg, -1);
} else {
if (!(actions & RD_KAFKA_ERR_ACTION_RETRY) &&
rkcg->rkcg_last_err != ErrorCode) {
/* Propagate non-retriable errors to the application */
rd_kafka_consumer_err(
rkcg->rkcg_q, rd_kafka_broker_id(rkb),
ErrorCode, 0, NULL, NULL,
RD_KAFKA_OFFSET_INVALID,
"FindCoordinator response error: %s", errstr);
/* Suppress repeated errors */
rkcg->rkcg_last_err = ErrorCode;
}
/* Retries are performed by the timer-intervalled
* coord queries, continue querying */
rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
}
rd_kafka_cgrp_serve(rkcg); /* Serve updated state, if possible */
}
/**
* Query for coordinator.
* Ask any broker in state UP
*
* Locality: main thread
*/
void rd_kafka_cgrp_coord_query (rd_kafka_cgrp_t *rkcg,
const char *reason) {
rd_kafka_broker_t *rkb;
rd_kafka_resp_err_t err;
rkb = rd_kafka_broker_any_usable(rkcg->rkcg_rk,
RD_POLL_NOWAIT,
RD_DO_LOCK,
RD_KAFKA_FEATURE_BROKER_GROUP_COORD,
"coordinator query");
if (!rkb) {
/* Reset the interval because there were no brokers. When a
* broker becomes available, we want to query it immediately. */
rd_interval_reset(&rkcg->rkcg_coord_query_intvl);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPQUERY",
"Group \"%.*s\": "
"no broker available for coordinator query: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), reason);
return;
}
rd_rkb_dbg(rkb, CGRP, "CGRPQUERY",
"Group \"%.*s\": querying for coordinator: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), reason);
err = rd_kafka_FindCoordinatorRequest(
rkb, RD_KAFKA_COORD_GROUP, rkcg->rkcg_group_id->str,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_FindCoordinator, rkcg);
if (err) {
rd_rkb_dbg(rkb, CGRP, "CGRPQUERY",
"Group \"%.*s\": "
"unable to send coordinator query: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_err2str(err));
rd_kafka_broker_destroy(rkb);
return;
}
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_QUERY_COORD)
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_WAIT_COORD);
rd_kafka_broker_destroy(rkb);
/* Back off the next intervalled query since we just sent one. */
rd_interval_reset_to_now(&rkcg->rkcg_coord_query_intvl, 0);
}
/**
* @brief Mark the current coordinator as dead.
*
* @locality main thread
*/
void rd_kafka_cgrp_coord_dead (rd_kafka_cgrp_t *rkcg, rd_kafka_resp_err_t err,
const char *reason) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORD",
"Group \"%.*s\": "
"marking the coordinator (%"PRId32") dead: %s: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_coord_id, rd_kafka_err2str(err), reason);
rd_kafka_cgrp_coord_update(rkcg, -1);
/* Re-query for coordinator */
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
rd_kafka_cgrp_coord_query(rkcg, reason);
}
/**
* @returns a new reference to the current coordinator, if available, else NULL.
*
* @locality rdkafka main thread
* @locks_required none
* @locks_acquired none
*/
rd_kafka_broker_t *rd_kafka_cgrp_get_coord (rd_kafka_cgrp_t *rkcg) {
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP || !rkcg->rkcg_coord)
return NULL;
rd_kafka_broker_keep(rkcg->rkcg_coord);
return rkcg->rkcg_coord;
}
/**
* @brief cgrp handling of LeaveGroup responses
* @param opaque must be the cgrp handle.
* @locality rdkafka main thread (unless err==ERR__DESTROY)
*/
static void rd_kafka_cgrp_handle_LeaveGroup (rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = opaque;
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
if (err) {
ErrorCode = err;
goto err;
}
if (request->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
err:
if (ErrorCode)
rd_kafka_dbg(rkb->rkb_rk, CGRP, "LEAVEGROUP",
"LeaveGroup response error in state %s: %s",
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_err2str(ErrorCode));
else
rd_kafka_dbg(rkb->rkb_rk, CGRP, "LEAVEGROUP",
"LeaveGroup response received in state %s",
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
if (ErrorCode != RD_KAFKA_RESP_ERR__DESTROY) {
rd_assert(thrd_is_current(rk->rk_thread));
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WAIT_LEAVE;
rd_kafka_cgrp_try_terminate(rkcg);
}
return;
err_parse:
ErrorCode = rkbuf->rkbuf_err;
goto err;
}
static void rd_kafka_cgrp_leave (rd_kafka_cgrp_t *rkcg) {
char *member_id;
RD_KAFKAP_STR_DUPA(&member_id, rkcg->rkcg_member_id);
/* Leaving the group invalidates the member id, reset it
* now to avoid an ERR_UNKNOWN_MEMBER_ID on the next join. */
rd_kafka_cgrp_set_member_id(rkcg, "");
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE",
"Group \"%.*s\": leave (in state %s): "
"LeaveGroupRequest already in-transit",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
return;
}
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE",
"Group \"%.*s\": leave (in state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WAIT_LEAVE;
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_UP) {
rd_rkb_dbg(rkcg->rkcg_curr_coord, CONSUMER, "LEAVE",
"Leaving group");
rd_kafka_LeaveGroupRequest(rkcg->rkcg_coord,
rkcg->rkcg_group_id->str,
member_id,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_LeaveGroup,
rkcg);
} else
rd_kafka_cgrp_handle_LeaveGroup(rkcg->rkcg_rk,
rkcg->rkcg_coord,
RD_KAFKA_RESP_ERR__WAIT_COORD,
NULL, NULL, rkcg);
}
/**
* @brief Leave group, if desired.
*
* @returns true if a LeaveGroup was issued, else false.
*/
static rd_bool_t rd_kafka_cgrp_leave_maybe (rd_kafka_cgrp_t *rkcg) {
/* We were not instructed to leave in the first place. */
if (!(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE))
return rd_false;
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE;
/* Don't send Leave when termating with NO_CONSUMER_CLOSE flag */
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk))
return rd_false;
/* KIP-345: Static group members must not send a LeaveGroupRequest
* on termination. */
if (RD_KAFKA_CGRP_IS_STATIC_MEMBER(rkcg) &&
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)
return rd_false;
rd_kafka_cgrp_leave(rkcg);
return rd_true;
}
/**
* @brief Enqueues a rebalance op, delegating responsibility of calling
* incremental_assign / incremental_unassign to the application.
* If there is no rebalance handler configured, or the action
* should not be delegated to the application for some other
* reason, incremental_assign / incremental_unassign will be called
* automatically, immediately.
*
* @param rejoin whether or not to rejoin the group following completion
* of the incremental assign / unassign.
*
* @remarks does not take ownership of \p partitions.
*/
void
rd_kafka_rebalance_op_incr (rd_kafka_cgrp_t *rkcg,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *partitions,
rd_bool_t rejoin,
const char *reason) {
rd_kafka_error_t *error;
/* Flag to rejoin after completion of the incr_assign or incr_unassign,
if required. */
rkcg->rkcg_rebalance_rejoin = rejoin;
rd_kafka_wrlock(rkcg->rkcg_rk);
rkcg->rkcg_c.ts_rebalance = rd_clock();
rkcg->rkcg_c.rebalance_cnt++;
rd_kafka_wrunlock(rkcg->rkcg_rk);
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk) ||
rd_kafka_fatal_error_code(rkcg->rkcg_rk)) {
/* Total unconditional unassign in these cases */
rd_kafka_cgrp_unassign(rkcg);
/* Now serve the assignment to make updates */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
goto done;
}
rd_kafka_cgrp_set_join_state(
rkcg,
err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS ?
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL :
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL);
/* Schedule application rebalance callback/event if enabled */
if (rkcg->rkcg_rk->rk_conf.enabled_events & RD_KAFKA_EVENT_REBALANCE) {
rd_kafka_op_t *rko;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGN",
"Group \"%s\": delegating incremental %s of %d "
"partition(s) to application on queue %s: %s",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS ?
"revoke" : "assign", partitions->cnt,