-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
Copy pathrdkafka_mock_cgrp.c
1848 lines (1586 loc) · 70.7 KB
/
rdkafka_mock_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) 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.
*/
/**
* Mocks
*
*/
#include "rdkafka_int.h"
#include "rdbuf.h"
#include "rdkafka_mock_int.h"
static const char *rd_kafka_mock_cgrp_classic_state_names[] = {
"Empty", "Joining", "Syncing", "Rebalancing", "Up"};
static void
rd_kafka_mock_cgrp_classic_rebalance(rd_kafka_mock_cgrp_classic_t *mcgrp,
const char *reason);
static void rd_kafka_mock_cgrp_classic_member_destroy(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member);
static void
rd_kafka_mock_cgrp_classic_set_state(rd_kafka_mock_cgrp_classic_t *mcgrp,
unsigned int new_state,
const char *reason) {
if (mcgrp->state == new_state)
return;
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Mock consumer group %s with %d member(s) "
"changing state %s -> %s: %s",
mcgrp->id, mcgrp->member_cnt,
rd_kafka_mock_cgrp_classic_state_names[mcgrp->state],
rd_kafka_mock_cgrp_classic_state_names[new_state], reason);
mcgrp->state = new_state;
}
/**
* @brief Mark member as active (restart session timer)
*/
void rd_kafka_mock_cgrp_classic_member_active(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Marking mock consumer group member %s as active",
member->id);
member->ts_last_activity = rd_clock();
}
/**
* @brief Verify that the protocol request is valid in the current state.
*
* @param member may be NULL.
*/
rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_check_state(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member,
const rd_kafka_buf_t *request,
int32_t generation_id) {
int16_t ApiKey = request->rkbuf_reqhdr.ApiKey;
rd_bool_t has_generation_id = ApiKey == RD_KAFKAP_SyncGroup ||
ApiKey == RD_KAFKAP_Heartbeat ||
ApiKey == RD_KAFKAP_OffsetCommit;
if (has_generation_id && generation_id != mcgrp->generation_id)
return RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION;
if (ApiKey == RD_KAFKAP_OffsetCommit && !member)
return RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
switch (mcgrp->state) {
case RD_KAFKA_MOCK_CGRP_STATE_EMPTY:
if (ApiKey == RD_KAFKAP_JoinGroup)
return RD_KAFKA_RESP_ERR_NO_ERROR;
break;
case RD_KAFKA_MOCK_CGRP_STATE_JOINING:
if (ApiKey == RD_KAFKAP_JoinGroup ||
ApiKey == RD_KAFKAP_LeaveGroup)
return RD_KAFKA_RESP_ERR_NO_ERROR;
else
return RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS;
case RD_KAFKA_MOCK_CGRP_STATE_SYNCING:
if (ApiKey == RD_KAFKAP_SyncGroup ||
ApiKey == RD_KAFKAP_JoinGroup ||
ApiKey == RD_KAFKAP_LeaveGroup)
return RD_KAFKA_RESP_ERR_NO_ERROR;
else
return RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS;
case RD_KAFKA_MOCK_CGRP_STATE_REBALANCING:
if (ApiKey == RD_KAFKAP_JoinGroup ||
ApiKey == RD_KAFKAP_LeaveGroup ||
ApiKey == RD_KAFKAP_OffsetCommit)
return RD_KAFKA_RESP_ERR_NO_ERROR;
else
return RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS;
case RD_KAFKA_MOCK_CGRP_STATE_UP:
if (ApiKey == RD_KAFKAP_JoinGroup ||
ApiKey == RD_KAFKAP_LeaveGroup ||
ApiKey == RD_KAFKAP_Heartbeat ||
ApiKey == RD_KAFKAP_OffsetCommit)
return RD_KAFKA_RESP_ERR_NO_ERROR;
break;
}
return RD_KAFKA_RESP_ERR_INVALID_REQUEST;
}
/**
* @brief Set a member's assignment (from leader's SyncGroupRequest)
*/
void rd_kafka_mock_cgrp_classic_member_assignment_set(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member,
const rd_kafkap_bytes_t *Metadata) {
if (member->assignment) {
rd_assert(mcgrp->assignment_cnt > 0);
mcgrp->assignment_cnt--;
rd_kafkap_bytes_destroy(member->assignment);
member->assignment = NULL;
}
if (Metadata) {
mcgrp->assignment_cnt++;
member->assignment = rd_kafkap_bytes_copy(Metadata);
}
}
/**
* @brief Sync done (successfully) or failed, send responses back to members.
*/
static void
rd_kafka_mock_cgrp_classic_sync_done(rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_resp_err_t err) {
rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
rd_kafka_buf_t *resp;
if ((resp = member->resp)) {
member->resp = NULL;
rd_assert(resp->rkbuf_reqhdr.ApiKey ==
RD_KAFKAP_SyncGroup);
rd_kafka_buf_write_i16(resp, err); /* ErrorCode */
/* MemberState */
rd_kafka_buf_write_kbytes(
resp, !err ? member->assignment : NULL);
}
rd_kafka_mock_cgrp_classic_member_assignment_set(mcgrp, member,
NULL);
if (member->conn) {
rd_kafka_mock_connection_set_blocking(member->conn,
rd_false);
if (resp)
rd_kafka_mock_connection_send_response(
member->conn, resp);
} else if (resp) {
/* Member has disconnected. */
rd_kafka_buf_destroy(resp);
}
}
}
/**
* @brief Check if all members have sent SyncGroupRequests, if so, propagate
* assignment to members.
*/
static void
rd_kafka_mock_cgrp_classic_sync_check(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Mock consumer group %s: awaiting %d/%d syncing members "
"in state %s",
mcgrp->id, mcgrp->assignment_cnt, mcgrp->member_cnt,
rd_kafka_mock_cgrp_classic_state_names[mcgrp->state]);
if (mcgrp->assignment_cnt < mcgrp->member_cnt)
return;
rd_kafka_mock_cgrp_classic_sync_done(mcgrp, RD_KAFKA_RESP_ERR_NO_ERROR);
rd_kafka_mock_cgrp_classic_set_state(mcgrp, RD_KAFKA_MOCK_CGRP_STATE_UP,
"all members synced");
}
/**
* @brief Member has sent SyncGroupRequest and is waiting for a response,
* which will be sent when the all group member SyncGroupRequest are
* received.
*/
rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_sync_set(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member,
rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *resp) {
if (mcgrp->state != RD_KAFKA_MOCK_CGRP_STATE_SYNCING)
return RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS; /* FIXME */
rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
rd_assert(!member->resp);
member->resp = resp;
member->conn = mconn;
rd_kafka_mock_connection_set_blocking(member->conn, rd_true);
/* Check if all members now have an assignment, if so, send responses */
rd_kafka_mock_cgrp_classic_sync_check(mcgrp);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Member is explicitly leaving the group (through LeaveGroupRequest)
*/
rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_leave(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Member %s is leaving group %s", member->id, mcgrp->id);
rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "explicit member leave");
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Destroys/frees an array of protocols, including the array itself.
*/
void rd_kafka_mock_cgrp_classic_protos_destroy(
rd_kafka_mock_cgrp_classic_proto_t *protos,
int proto_cnt) {
int i;
for (i = 0; i < proto_cnt; i++) {
rd_free(protos[i].name);
if (protos[i].metadata)
rd_free(protos[i].metadata);
}
rd_free(protos);
}
static void rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
rd_kafka_mock_cgrp_classic_t *mcgrp,
int timeout_ms);
/**
* @brief Elect consumer group leader and send JoinGroup responses
*/
static void
rd_kafka_mock_cgrp_classic_elect_leader(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_mock_cgrp_classic_member_t *member;
rd_assert(mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_JOINING);
rd_assert(!TAILQ_EMPTY(&mcgrp->members));
mcgrp->generation_id++;
/* Elect a leader deterministically if the group.instance.id is
* available, using the lexicographic order of group.instance.ids.
* This is not how it's done on a real broker, which uses the first
* member joined. But we use a determinstic method for better testing,
* (in case we want to enforce a some consumer to be the group leader).
* If group.instance.id is not specified for any consumer, we use the
* first one joined, similar to the real broker. */
mcgrp->leader = NULL;
TAILQ_FOREACH(member, &mcgrp->members, link) {
if (!mcgrp->leader)
mcgrp->leader = member;
else if (mcgrp->leader->group_instance_id &&
member->group_instance_id &&
(rd_strcmp(mcgrp->leader->group_instance_id,
member->group_instance_id) > 0))
mcgrp->leader = member;
}
rd_kafka_dbg(
mcgrp->cluster->rk, MOCK, "MOCK",
"Consumer group %s with %d member(s) is rebalancing: "
"elected leader is %s (group.instance.id = %s), generation id %d",
mcgrp->id, mcgrp->member_cnt, mcgrp->leader->id,
mcgrp->leader->group_instance_id, mcgrp->generation_id);
/* Find the most commonly supported protocol name among the members.
* FIXME: For now we'll blindly use the first protocol of the leader. */
if (mcgrp->protocol_name)
rd_free(mcgrp->protocol_name);
mcgrp->protocol_name = RD_KAFKAP_STR_DUP(mcgrp->leader->protos[0].name);
/* Send JoinGroupResponses to all members */
TAILQ_FOREACH(member, &mcgrp->members, link) {
rd_bool_t is_leader = member == mcgrp->leader;
int member_cnt = is_leader ? mcgrp->member_cnt : 0;
rd_kafka_buf_t *resp;
rd_kafka_mock_cgrp_classic_member_t *member2;
rd_kafka_mock_connection_t *mconn;
/* Member connection has been closed, it will eventually
* reconnect or time out from the group. */
if (!member->conn || !member->resp)
continue;
mconn = member->conn;
member->conn = NULL;
resp = member->resp;
member->resp = NULL;
rd_assert(resp->rkbuf_reqhdr.ApiKey == RD_KAFKAP_JoinGroup);
rd_kafka_buf_write_i16(resp, 0); /* ErrorCode */
rd_kafka_buf_write_i32(resp, mcgrp->generation_id);
rd_kafka_buf_write_str(resp, mcgrp->protocol_name, -1);
rd_kafka_buf_write_str(resp, mcgrp->leader->id, -1);
rd_kafka_buf_write_str(resp, member->id, -1);
rd_kafka_buf_write_i32(resp, member_cnt);
/* Send full member list to leader */
if (member_cnt > 0) {
TAILQ_FOREACH(member2, &mcgrp->members, link) {
rd_kafka_buf_write_str(resp, member2->id, -1);
if (resp->rkbuf_reqhdr.ApiVersion >= 5)
rd_kafka_buf_write_str(
resp, member2->group_instance_id,
-1);
/* FIXME: look up correct protocol name */
rd_assert(!rd_kafkap_str_cmp_str(
member2->protos[0].name,
mcgrp->protocol_name));
rd_kafka_buf_write_kbytes(
resp, member2->protos[0].metadata);
}
}
/* Mark each member as active to avoid them timing out
* at the same time as a JoinGroup handler that blocks
* session.timeout.ms to elect a leader. */
rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
rd_kafka_mock_connection_set_blocking(mconn, rd_false);
rd_kafka_mock_connection_send_response(mconn, resp);
}
mcgrp->last_member_cnt = mcgrp->member_cnt;
rd_kafka_mock_cgrp_classic_set_state(mcgrp,
RD_KAFKA_MOCK_CGRP_STATE_SYNCING,
"leader elected, waiting for all "
"members to sync");
rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
mcgrp, mcgrp->session_timeout_ms);
}
/**
* @brief Trigger group rebalance.
*/
static void
rd_kafka_mock_cgrp_classic_rebalance(rd_kafka_mock_cgrp_classic_t *mcgrp,
const char *reason) {
int timeout_ms;
if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_JOINING)
return; /* Do nothing, group is already rebalancing. */
else if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_EMPTY)
/* First join, low timeout.
* Same as group.initial.rebalance.delay.ms
* on the broker. */
timeout_ms =
mcgrp->cluster->defaults.group_initial_rebalance_delay_ms;
else if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_REBALANCING &&
mcgrp->member_cnt == mcgrp->last_member_cnt)
timeout_ms = 100; /* All members rejoined, quickly transition
* to election. */
else /* Let the rebalance delay be a bit shorter than the
* session timeout so that we don't time out waiting members
* who are also subject to the session timeout. */
timeout_ms = mcgrp->session_timeout_ms > 1000
? mcgrp->session_timeout_ms - 1000
: mcgrp->session_timeout_ms;
if (mcgrp->state == RD_KAFKA_MOCK_CGRP_STATE_SYNCING)
/* Abort current Syncing state */
rd_kafka_mock_cgrp_classic_sync_done(
mcgrp, RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS);
rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_JOINING, reason);
rd_kafka_mock_cgrp_classic_rebalance_timer_restart(mcgrp, timeout_ms);
}
/**
* @brief Consumer group state machine triggered by timer events.
*/
static void
rd_kafka_mock_cgrp_classic_fsm_timeout(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Mock consumer group %s FSM timeout in state %s",
mcgrp->id,
rd_kafka_mock_cgrp_classic_state_names[mcgrp->state]);
switch (mcgrp->state) {
case RD_KAFKA_MOCK_CGRP_STATE_EMPTY:
/* No members, do nothing */
break;
case RD_KAFKA_MOCK_CGRP_STATE_JOINING:
/* Timed out waiting for more members, elect a leader */
if (mcgrp->member_cnt > 0)
rd_kafka_mock_cgrp_classic_elect_leader(mcgrp);
else
rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_EMPTY,
"no members joined");
break;
case RD_KAFKA_MOCK_CGRP_STATE_SYNCING:
/* Timed out waiting for all members to sync */
/* Send error response to all waiting members */
rd_kafka_mock_cgrp_classic_sync_done(
mcgrp, RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS /* FIXME */);
rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_REBALANCING,
"timed out waiting for all members to synchronize");
break;
case RD_KAFKA_MOCK_CGRP_STATE_REBALANCING:
/* Timed out waiting for all members to Leave or re-Join */
rd_kafka_mock_cgrp_classic_set_state(
mcgrp, RD_KAFKA_MOCK_CGRP_STATE_JOINING,
"timed out waiting for all "
"members to re-Join or Leave");
break;
case RD_KAFKA_MOCK_CGRP_STATE_UP:
/* No fsm timers triggered in this state, see
* the session_tmr instead */
break;
}
}
static void rd_kafka_mcgrp_rebalance_timer_cb(rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_mock_cgrp_classic_t *mcgrp = arg;
rd_kafka_mock_cgrp_classic_fsm_timeout(mcgrp);
}
/**
* @brief Restart the rebalance timer, postponing leader election.
*/
static void rd_kafka_mock_cgrp_classic_rebalance_timer_restart(
rd_kafka_mock_cgrp_classic_t *mcgrp,
int timeout_ms) {
rd_kafka_timer_start_oneshot(
&mcgrp->cluster->timers, &mcgrp->rebalance_tmr, rd_true,
timeout_ms * 1000, rd_kafka_mcgrp_rebalance_timer_cb, mcgrp);
}
static void rd_kafka_mock_cgrp_classic_member_destroy(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member) {
rd_assert(mcgrp->member_cnt > 0);
TAILQ_REMOVE(&mcgrp->members, member, link);
mcgrp->member_cnt--;
rd_free(member->id);
if (member->resp)
rd_kafka_buf_destroy(member->resp);
if (member->group_instance_id)
rd_free(member->group_instance_id);
rd_kafka_mock_cgrp_classic_member_assignment_set(mcgrp, member, NULL);
rd_kafka_mock_cgrp_classic_protos_destroy(member->protos,
member->proto_cnt);
rd_free(member);
}
/**
* @brief Find member in group.
*/
rd_kafka_mock_cgrp_classic_member_t *rd_kafka_mock_cgrp_classic_member_find(
const rd_kafka_mock_cgrp_classic_t *mcgrp,
const rd_kafkap_str_t *MemberId) {
const rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
if (!rd_kafkap_str_cmp_str(MemberId, member->id))
return (rd_kafka_mock_cgrp_classic_member_t *)member;
}
return NULL;
}
/**
* @brief Update or add member to consumer group
*/
rd_kafka_resp_err_t rd_kafka_mock_cgrp_classic_member_add(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_t *resp,
const rd_kafkap_str_t *MemberId,
const rd_kafkap_str_t *ProtocolType,
const rd_kafkap_str_t *GroupInstanceId,
rd_kafka_mock_cgrp_classic_proto_t *protos,
int proto_cnt,
int session_timeout_ms) {
rd_kafka_mock_cgrp_classic_member_t *member;
rd_kafka_resp_err_t err;
err = rd_kafka_mock_cgrp_classic_check_state(mcgrp, NULL, resp, -1);
if (err)
return err;
/* Find member */
member = rd_kafka_mock_cgrp_classic_member_find(mcgrp, MemberId);
if (!member) {
/* Not found, add member */
member = rd_calloc(1, sizeof(*member));
if (!RD_KAFKAP_STR_LEN(MemberId)) {
/* Generate a member id */
char memberid[32];
rd_snprintf(memberid, sizeof(memberid), "%p", member);
member->id = rd_strdup(memberid);
} else
member->id = RD_KAFKAP_STR_DUP(MemberId);
if (RD_KAFKAP_STR_LEN(GroupInstanceId))
member->group_instance_id =
RD_KAFKAP_STR_DUP(GroupInstanceId);
TAILQ_INSERT_TAIL(&mcgrp->members, member, link);
mcgrp->member_cnt++;
}
if (mcgrp->state != RD_KAFKA_MOCK_CGRP_STATE_JOINING)
rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "member join");
mcgrp->session_timeout_ms = session_timeout_ms;
if (member->protos)
rd_kafka_mock_cgrp_classic_protos_destroy(member->protos,
member->proto_cnt);
member->protos = protos;
member->proto_cnt = proto_cnt;
rd_assert(!member->resp);
member->resp = resp;
member->conn = mconn;
rd_kafka_mock_cgrp_classic_member_active(mcgrp, member);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Check if any members have exceeded the session timeout.
*/
static void rd_kafka_mock_cgrp_classic_session_tmr_cb(rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_mock_cgrp_classic_t *mcgrp = arg;
rd_kafka_mock_cgrp_classic_member_t *member, *tmp;
rd_ts_t now = rd_clock();
int timeout_cnt = 0;
TAILQ_FOREACH_SAFE(member, &mcgrp->members, link, tmp) {
if (member->ts_last_activity +
(mcgrp->session_timeout_ms * 1000) >
now)
continue;
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Member %s session timed out for group %s",
member->id, mcgrp->id);
rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
timeout_cnt++;
}
if (timeout_cnt)
rd_kafka_mock_cgrp_classic_rebalance(mcgrp, "member timeout");
}
void rd_kafka_mock_cgrp_classic_destroy(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_REMOVE(&mcgrp->cluster->cgrps_classic, mcgrp, link);
rd_kafka_timer_stop(&mcgrp->cluster->timers, &mcgrp->rebalance_tmr,
rd_true);
rd_kafka_timer_stop(&mcgrp->cluster->timers, &mcgrp->session_tmr,
rd_true);
rd_free(mcgrp->id);
rd_free(mcgrp->protocol_type);
if (mcgrp->protocol_name)
rd_free(mcgrp->protocol_name);
while ((member = TAILQ_FIRST(&mcgrp->members)))
rd_kafka_mock_cgrp_classic_member_destroy(mcgrp, member);
rd_free(mcgrp);
}
rd_kafka_mock_cgrp_classic_t *
rd_kafka_mock_cgrp_classic_find(rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId) {
rd_kafka_mock_cgrp_classic_t *mcgrp;
TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
return mcgrp;
}
return NULL;
}
/**
* @brief Find or create a classic consumer group
*/
rd_kafka_mock_cgrp_classic_t *
rd_kafka_mock_cgrp_classic_get(rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId,
const rd_kafkap_str_t *ProtocolType) {
rd_kafka_mock_cgrp_classic_t *mcgrp;
mcgrp = rd_kafka_mock_cgrp_classic_find(mcluster, GroupId);
if (mcgrp)
return mcgrp;
/* FIXME: What to do with mismatching ProtocolTypes? */
mcgrp = rd_calloc(1, sizeof(*mcgrp));
mcgrp->cluster = mcluster;
mcgrp->id = RD_KAFKAP_STR_DUP(GroupId);
mcgrp->protocol_type = RD_KAFKAP_STR_DUP(ProtocolType);
mcgrp->generation_id = 1;
TAILQ_INIT(&mcgrp->members);
rd_kafka_timer_start(&mcluster->timers, &mcgrp->session_tmr,
1000 * 1000 /*1s*/,
rd_kafka_mock_cgrp_classic_session_tmr_cb, mcgrp);
TAILQ_INSERT_TAIL(&mcluster->cgrps_classic, mcgrp, link);
return mcgrp;
}
/**
* @brief A client connection closed, check if any classic cgrp has any state
* for this connection that needs to be cleared.
*/
void rd_kafka_mock_cgrps_classic_connection_closed(
rd_kafka_mock_cluster_t *mcluster,
rd_kafka_mock_connection_t *mconn) {
rd_kafka_mock_cgrp_classic_t *mcgrp;
TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
rd_kafka_mock_cgrp_classic_member_t *member, *tmp;
TAILQ_FOREACH_SAFE(member, &mcgrp->members, link, tmp) {
if (member->conn == mconn) {
member->conn = NULL;
if (member->resp) {
rd_kafka_buf_destroy(member->resp);
member->resp = NULL;
}
}
}
}
}
/**
* @struct Target assignment for a consumer group.
* `member_ids` and `assignment` are in the same order
* and have the same count.
*/
typedef struct rd_kafka_mock_cgrp_consumer_target_assignment_s {
rd_list_t *member_ids; /**< Member id list (char *). */
rd_list_t *assignment; /**< Assingment list
(rd_kafka_topic_partition_list_t *). */
} rd_kafka_mock_cgrp_consumer_target_assignment_t;
static rd_kafka_mock_cgrp_consumer_target_assignment_t *
rd_kafka_mock_cgrp_consumer_target_assignment_new0(rd_list_t *member_ids,
rd_list_t *assignment) {
rd_assert(member_ids->rl_cnt == assignment->rl_cnt);
rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment =
rd_calloc(1, sizeof(*target_assignment));
target_assignment->member_ids =
rd_list_copy(member_ids, rd_list_string_copy, NULL);
target_assignment->assignment = rd_list_copy(
assignment, rd_kafka_topic_partition_list_copy_opaque, NULL);
return target_assignment;
}
rd_kafka_mock_cgrp_consumer_target_assignment_t *
rd_kafka_mock_cgrp_consumer_target_assignment_new(
char **member_ids,
int member_cnt,
rd_kafka_topic_partition_list_t **assignment) {
int i;
rd_list_t *member_id_list, *assignment_list;
rd_kafka_mock_cgrp_consumer_target_assignment_t *ret;
member_id_list = rd_list_new(member_cnt, rd_free);
assignment_list =
rd_list_new(member_cnt, rd_kafka_topic_partition_list_destroy_free);
for (i = 0; i < member_cnt; i++) {
rd_list_add(member_id_list, rd_strdup(member_ids[i]));
rd_list_add(assignment_list,
rd_kafka_topic_partition_list_copy(assignment[i]));
}
ret = rd_kafka_mock_cgrp_consumer_target_assignment_new0(
member_id_list, assignment_list);
rd_list_destroy(member_id_list);
rd_list_destroy(assignment_list);
return ret;
}
void rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
rd_list_destroy(target_assignment->member_ids);
rd_list_destroy(target_assignment->assignment);
rd_free(target_assignment);
}
/**
* @brief Sets next target assignment and member epoch for \p member
* to a copy of partition list \p rktparlist,
* filling its topic ids if not provided, using \p cgrp cluster topics.
*
* @param mcgrp The consumer group containing the member.
* @param member A consumer group member.
* @param target_member_epoch New member epoch.
* @param rktparlist Next target assignment.
*
* @locks mcluster->lock MUST be held.
*/
static void rd_kafka_mock_cgrp_consumer_member_target_assignment_set(
rd_kafka_mock_cgrp_consumer_t *mcgrp,
rd_kafka_mock_cgrp_consumer_member_t *member,
int target_member_epoch,
const rd_kafka_topic_partition_list_t *rktparlist) {
rd_kafka_topic_partition_t *rktpar;
if (member->target_assignment) {
rd_kafka_topic_partition_list_destroy(
member->target_assignment);
}
member->target_member_epoch = target_member_epoch;
member->target_assignment =
rd_kafka_topic_partition_list_copy(rktparlist);
/* If not present, fill topic ids using names */
RD_KAFKA_TPLIST_FOREACH(rktpar, member->target_assignment) {
rd_kafka_Uuid_t topic_id =
rd_kafka_topic_partition_get_topic_id(rktpar);
if (!rd_kafka_Uuid_cmp(topic_id, RD_KAFKA_UUID_ZERO)) {
rd_kafka_mock_topic_t *mtopic =
rd_kafka_mock_topic_find(mcgrp->cluster,
rktpar->topic);
if (mtopic)
rd_kafka_topic_partition_set_topic_id(
rktpar, mtopic->id);
}
}
}
/**
* @brief Sets next target assignment for group \p mcgrp
* to a copy of \p target_assignment partition lists.
*
* @param mcgrp The consumer group.
* @param target_assignment Target assignment for all members.
*
* @locks mcluster->lock MUST be held.
*/
static void rd_kafka_mock_cgrp_consumer_target_assignment_set(
rd_kafka_mock_cgrp_consumer_t *mcgrp,
rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {
int i = 0;
int32_t new_target_member_epoch;
const char *member_id;
rd_kafka_mock_cgrp_consumer_member_t *member;
mcgrp->group_epoch++;
new_target_member_epoch = mcgrp->group_epoch;
RD_LIST_FOREACH(member_id, target_assignment->member_ids, i) {
rd_kafkap_str_t *member_id_str =
rd_kafkap_str_new(member_id, strlen(member_id));
rd_kafka_topic_partition_list_t *member_assignment =
rd_list_elem(target_assignment->assignment, i);
member = rd_kafka_mock_cgrp_consumer_member_find(mcgrp,
member_id_str);
rd_kafkap_str_destroy(member_id_str);
if (!member)
continue;
rd_kafka_mock_cgrp_consumer_member_target_assignment_set(
mcgrp, member, new_target_member_epoch, member_assignment);
}
}
typedef RD_MAP_TYPE(const char *, rd_list_t *) map_str_list;
typedef RD_MAP_TYPE(const char *, int *) map_str_int;
/**
* @brief Calculate a simple range target assignment for the consumer group \p
* mcgrp. This isn't replicating any given broker assignor but is used
* when the test doesn't need a specific type of assignment.
*
* If the test needs it, instead of replicating same conditions with all the
* members, one can mock the assignment directly with
* `rd_kafka_mock_cgrp_consumer_target_assignment`.
*/
static rd_kafka_mock_cgrp_consumer_target_assignment_t *
rd_kafka_mock_cgrp_consumer_target_assignment_calculate_range(
const rd_kafka_mock_cgrp_consumer_t *mcgrp) {
int i, *i_pointer;
const char *topic;
rd_list_t *members;
rd_kafka_mock_cgrp_consumer_member_t *member;
rd_kafka_mock_cluster_t *mcluster = mcgrp->cluster;
/* List of member ids (char *) */
rd_list_t *member_ids = rd_list_new(mcgrp->member_cnt, rd_free);
/* List of member assignment (rd_kafka_topic_partition_list_t *) */
rd_list_t *assignment = rd_list_new(
mcgrp->member_cnt, rd_kafka_topic_partition_list_destroy_free);
/* Map from topic name to list of members */
map_str_list topic_members =
RD_MAP_INITIALIZER(mcgrp->member_cnt, rd_map_str_cmp,
rd_map_str_hash, NULL, rd_list_destroy_free);
/* Map from member id to index in the members and assignment lists. */
map_str_int member_idx = RD_MAP_INITIALIZER(
mcgrp->member_cnt, rd_map_str_cmp, rd_map_str_hash, NULL, rd_free);
i = 0;
/* First create a map with topics associated to the list of members
* and save the member idx in the `member_idx` map. */
TAILQ_FOREACH(member, &mcgrp->members, link) {
int j;
rd_list_add(member_ids, rd_strdup(member->id));
rd_list_add(assignment, rd_kafka_topic_partition_list_new(0));
RD_LIST_FOREACH(topic, member->subscribed_topics, j) {
if (!RD_MAP_GET(&topic_members, topic)) {
members = rd_list_new(0, NULL);
RD_MAP_SET(&topic_members, topic, members);
} else
members = RD_MAP_GET(&topic_members, topic);
rd_list_add(members, member);
}
i_pointer = rd_calloc(1, sizeof(*i_pointer));
*i_pointer = i;
RD_MAP_SET(&member_idx, member->id, i_pointer);
i++;
}
/* For each topic to a range assignment and add the
* corresponding partitions to the assignment for that member.
* Finds the list index using the `member_idx` map. */
RD_MAP_FOREACH(topic, members, &topic_members) {
rd_kafka_Uuid_t topic_id;
rd_kafka_topic_partition_list_t *member_assignment;
int members_cnt = rd_list_cnt(members);
int common, one_more, assigned = 0;
rd_kafkap_str_t Topic = {.str = topic, .len = strlen(topic)};
rd_kafka_mock_topic_t *mock_topic =
rd_kafka_mock_topic_find_by_kstr(mcluster, &Topic);
if (!mock_topic)
continue;
topic_id = mock_topic->id;
/* Assign one partition more
* to the first mock_topic->partition_cnt % members_cnt
* members. */
common = mock_topic->partition_cnt / members_cnt;
one_more = mock_topic->partition_cnt % members_cnt;
RD_LIST_FOREACH(member, members, i) {
int j, num_partitions = common;
int idx = *RD_MAP_GET(&member_idx, member->id);
member_assignment = rd_list_elem(assignment, idx);
if (idx < one_more)
num_partitions++;
for (j = 0; j < num_partitions; j++) {
rd_kafka_topic_partition_t *rktpar =
rd_kafka_topic_partition_list_add(
member_assignment, topic, assigned + j);
rd_kafka_topic_partition_set_topic_id(rktpar,
topic_id);
}
assigned += num_partitions;
}
}
rd_kafka_mock_cgrp_consumer_target_assignment_t *ret =
rd_kafka_mock_cgrp_consumer_target_assignment_new0(member_ids,
assignment);
RD_MAP_DESTROY(&topic_members);
RD_MAP_DESTROY(&member_idx);
rd_list_destroy(member_ids);
rd_list_destroy(assignment);
return ret;
}
/**
* @brief Recalculate and set a target assignment for \p mcgrp
* only if `mcgrp->manual_assignment` isn't set.
*
* @locks mcluster->lock MUST be held.
*/
static void rd_kafka_mock_cgrp_consumer_target_assignment_recalculate(
rd_kafka_mock_cgrp_consumer_t *mcgrp) {
if (mcgrp->manual_assignment)
return;
rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment =
rd_kafka_mock_cgrp_consumer_target_assignment_calculate_range(
mcgrp);
rd_kafka_mock_cgrp_consumer_target_assignment_set(mcgrp,
target_assignment);
rd_kafka_mock_cgrp_consumer_target_assignment_destroy(
target_assignment);
}
/**
* @brief Set manual target assignment \p target_assignment
* to the consumer group \p mcgrp .
*
* @param mcgrp Consumer group
* @param target_assignment Target assignment to set.
* Pass NULL to return to automatic assignment.
*
* @locks mcluster->lock MUST be held.
*/
static void rd_kafka_mock_cgrp_consumer_target_assignment_set_manual(
rd_kafka_mock_cgrp_consumer_t *mcgrp,
rd_kafka_mock_cgrp_consumer_target_assignment_t *target_assignment) {