@@ -263,11 +263,28 @@ typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
263
263
264
264
/**
265
265
* @returns true if consumer has joined the group and thus requires a leave.
266
+ *
267
+ * `rkcg_member_id` is sufficient to know this with "classic" group protocol.
266
268
*/
267
- #define RD_KAFKA_CGRP_HAS_JOINED (rkcg ) \
268
- (rkcg->rkcg_member_id != NULL && \
269
+ #define RD_KAFKA_CGRP_HAS_JOINED_CLASSIC (rkcg ) \
270
+ (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC && \
271
+ rkcg->rkcg_member_id != NULL && \
269
272
RD_KAFKAP_STR_LEN((rkcg)->rkcg_member_id) > 0)
270
273
274
+ /**
275
+ * @returns true if consumer has joined the group and thus requires a leave.
276
+ *
277
+ * With "consumer" group protocol we cannot rely on the `rkcg_member_id`
278
+ * as it's client generated.
279
+ */
280
+ #define RD_KAFKA_CGRP_HAS_JOINED_CONSUMER (rkcg ) \
281
+ (rkcg->rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER && \
282
+ rkcg->rkcg_generation_id > 0)
283
+
284
+ #define RD_KAFKA_CGRP_HAS_JOINED (rkcg ) \
285
+ (RD_KAFKA_CGRP_HAS_JOINED_CLASSIC(rkcg) || \
286
+ RD_KAFKA_CGRP_HAS_JOINED_CONSUMER(rkcg))
287
+
271
288
272
289
/**
273
290
* @returns true if cgrp is waiting for a rebalance_cb to be handled by
@@ -384,6 +401,8 @@ void rd_kafka_cgrp_set_join_state(rd_kafka_cgrp_t *rkcg, int join_state) {
384
401
void rd_kafka_cgrp_destroy_final (rd_kafka_cgrp_t * rkcg ) {
385
402
rd_kafka_assert (rkcg -> rkcg_rk , !rkcg -> rkcg_subscription );
386
403
rd_kafka_assert (rkcg -> rkcg_rk , !rkcg -> rkcg_group_leader .members );
404
+ rd_kafka_assert (rkcg -> rkcg_rk , !rkcg -> rkcg_subscription_topics );
405
+ rd_kafka_assert (rkcg -> rkcg_rk , !rkcg -> rkcg_subscription_regex );
387
406
rd_kafka_cgrp_set_member_id (rkcg , NULL );
388
407
rd_kafka_topic_partition_list_destroy (rkcg -> rkcg_current_assignment );
389
408
RD_IF_FREE (rkcg -> rkcg_target_assignment ,
@@ -466,7 +485,15 @@ rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk,
466
485
rkcg -> rkcg_next_subscription = NULL ;
467
486
TAILQ_INIT (& rkcg -> rkcg_topics );
468
487
rd_list_init (& rkcg -> rkcg_toppars , 32 , NULL );
469
- rd_kafka_cgrp_set_member_id (rkcg , "" );
488
+
489
+ if (rkcg -> rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CONSUMER ) {
490
+ rd_kafka_Uuid_t uuid = rd_kafka_Uuid_random ();
491
+ rd_kafka_cgrp_set_member_id (rkcg ,
492
+ rd_kafka_Uuid_base64str (& uuid ));
493
+ } else {
494
+ rd_kafka_cgrp_set_member_id (rkcg , "" );
495
+ }
496
+
470
497
rkcg -> rkcg_subscribed_topics =
471
498
rd_list_new (0 , (void * )rd_kafka_topic_info_destroy );
472
499
rd_interval_init (& rkcg -> rkcg_coord_query_intvl );
@@ -1010,7 +1037,9 @@ static void rd_kafka_cgrp_consumer_leave(rd_kafka_cgrp_t *rkcg) {
1010
1037
rkcg -> rkcg_coord , rkcg -> rkcg_group_id , rkcg -> rkcg_member_id ,
1011
1038
member_epoch , rkcg -> rkcg_group_instance_id ,
1012
1039
NULL /* no rack */ , -1 /* no rebalance_timeout_ms */ ,
1013
- NULL /* no subscription */ , NULL /* no remote assignor */ ,
1040
+ NULL /* no subscription topics */ ,
1041
+ NULL /* no regex subscription */ ,
1042
+ NULL /* no remote assignor */ ,
1014
1043
NULL /* no current assignment */ ,
1015
1044
RD_KAFKA_REPLYQ (rkcg -> rkcg_ops , 0 ),
1016
1045
rd_kafka_cgrp_handle_ConsumerGroupHeartbeat_leave , rkcg );
@@ -1339,7 +1368,6 @@ static void rd_kafka_rebalance_op(rd_kafka_cgrp_t *rkcg,
1339
1368
rd_kafka_cgrp_group_assignment_set (rkcg , NULL );
1340
1369
}
1341
1370
1342
-
1343
1371
/**
1344
1372
* @brief Rejoin the group.
1345
1373
*
@@ -2942,14 +2970,15 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk,
2942
2970
}
2943
2971
2944
2972
rd_kafka_buf_read_str (rkbuf , & member_id );
2973
+ if (!RD_KAFKAP_STR_IS_NULL (& member_id )) {
2974
+ rd_kafka_cgrp_set_member_id (rkcg , member_id .str );
2975
+ }
2976
+
2945
2977
rd_kafka_buf_read_i32 (rkbuf , & member_epoch );
2946
2978
rd_kafka_buf_read_i32 (rkbuf , & heartbeat_interval_ms );
2947
2979
2948
2980
int8_t are_assignments_present ;
2949
2981
rd_kafka_buf_read_i8 (rkbuf , & are_assignments_present );
2950
- if (!RD_KAFKAP_STR_IS_NULL (& member_id )) {
2951
- rd_kafka_cgrp_set_member_id (rkcg , member_id .str );
2952
- }
2953
2982
rkcg -> rkcg_generation_id = member_epoch ;
2954
2983
if (heartbeat_interval_ms > 0 ) {
2955
2984
rkcg -> rkcg_heartbeat_intvl_ms = heartbeat_interval_ms ;
@@ -5171,12 +5200,46 @@ rd_kafka_cgrp_calculate_subscribe_revoking_partitions(
5171
5200
static int32_t
5172
5201
rd_kafka_cgrp_subscription_set (rd_kafka_cgrp_t * rkcg ,
5173
5202
rd_kafka_topic_partition_list_t * rktparlist ) {
5174
- int32_t new_subscription_version =
5175
- rd_atomic32_add (& rkcg -> rkcg_subscription_version , 1 );
5203
+
5204
+ rkcg -> rkcg_flags &= ~(RD_KAFKA_CGRP_F_SUBSCRIPTION |
5205
+ RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION );
5206
+
5207
+ RD_IF_FREE (rkcg -> rkcg_subscription ,
5208
+ rd_kafka_topic_partition_list_destroy );
5209
+ RD_IF_FREE (rkcg -> rkcg_subscription_topics ,
5210
+ rd_kafka_topic_partition_list_destroy );
5211
+ RD_IF_FREE (rkcg -> rkcg_subscription_regex , rd_kafkap_str_destroy );
5212
+
5176
5213
rkcg -> rkcg_subscription = rktparlist ;
5177
- return new_subscription_version ;
5214
+
5215
+ if (rkcg -> rkcg_subscription ) {
5216
+ rkcg -> rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION ;
5217
+ if (rd_kafka_topic_partition_list_regex_cnt (
5218
+ rkcg -> rkcg_subscription ) > 0 )
5219
+ rkcg -> rkcg_flags |=
5220
+ RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
5221
+
5222
+ if (rkcg -> rkcg_group_protocol ==
5223
+ RD_KAFKA_GROUP_PROTOCOL_CONSUMER ) {
5224
+ rkcg -> rkcg_subscription_regex =
5225
+ rd_kafka_topic_partition_list_combine_regexes (
5226
+ rkcg -> rkcg_subscription );
5227
+ rkcg -> rkcg_subscription_topics =
5228
+ rd_kafka_topic_partition_list_remove_regexes (
5229
+ rkcg -> rkcg_subscription );
5230
+ rkcg -> rkcg_consumer_flags |=
5231
+ RD_KAFKA_CGRP_CONSUMER_F_SUBSCRIBED_ONCE |
5232
+ RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION ;
5233
+ }
5234
+ } else {
5235
+ rkcg -> rkcg_subscription_regex = NULL ;
5236
+ rkcg -> rkcg_subscription_topics = NULL ;
5237
+ }
5238
+
5239
+ return rd_atomic32_add (& rkcg -> rkcg_subscription_version , 1 );
5178
5240
}
5179
5241
5242
+
5180
5243
/**
5181
5244
* @brief Handle a new subscription that is modifying an existing subscription
5182
5245
* in the COOPERATIVE case.
@@ -5194,11 +5257,6 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
5194
5257
int old_cnt = rkcg -> rkcg_subscription -> cnt ;
5195
5258
int32_t cgrp_subscription_version ;
5196
5259
5197
- rkcg -> rkcg_flags &= ~RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
5198
-
5199
- if (rd_kafka_topic_partition_list_regex_cnt (rktparlist ) > 0 )
5200
- rkcg -> rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
5201
-
5202
5260
/* Topics in rkcg_subscribed_topics that don't match any pattern in
5203
5261
the new subscription. */
5204
5262
unsubscribing_topics =
@@ -5208,7 +5266,6 @@ rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
5208
5266
revoking = rd_kafka_cgrp_calculate_subscribe_revoking_partitions (
5209
5267
rkcg , unsubscribing_topics );
5210
5268
5211
- rd_kafka_topic_partition_list_destroy (rkcg -> rkcg_subscription );
5212
5269
cgrp_subscription_version =
5213
5270
rd_kafka_cgrp_subscription_set (rkcg , rktparlist );
5214
5271
@@ -5318,10 +5375,7 @@ static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
5318
5375
rd_kafka_timer_stop (& rkcg -> rkcg_rk -> rk_timers ,
5319
5376
& rkcg -> rkcg_max_poll_interval_tmr , 1 /*lock*/ );
5320
5377
5321
- if (rkcg -> rkcg_subscription ) {
5322
- rd_kafka_topic_partition_list_destroy (rkcg -> rkcg_subscription );
5323
- rd_kafka_cgrp_subscription_set (rkcg , NULL );
5324
- }
5378
+ rd_kafka_cgrp_subscription_set (rkcg , NULL );
5325
5379
5326
5380
if (rkcg -> rkcg_group_protocol == RD_KAFKA_GROUP_PROTOCOL_CLASSIC )
5327
5381
rd_kafka_cgrp_update_subscribed_topics (rkcg , NULL );
@@ -5340,9 +5394,6 @@ static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
5340
5394
rd_true /*initiating*/ ,
5341
5395
"unsubscribe" );
5342
5396
5343
- rkcg -> rkcg_flags &= ~(RD_KAFKA_CGRP_F_SUBSCRIPTION |
5344
- RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION );
5345
-
5346
5397
return RD_KAFKA_RESP_ERR_NO_ERROR ;
5347
5398
}
5348
5399
@@ -5413,11 +5464,6 @@ rd_kafka_cgrp_subscribe(rd_kafka_cgrp_t *rkcg,
5413
5464
if (!rktparlist )
5414
5465
return RD_KAFKA_RESP_ERR_NO_ERROR ;
5415
5466
5416
- rkcg -> rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION ;
5417
-
5418
- if (rd_kafka_topic_partition_list_regex_cnt (rktparlist ) > 0 )
5419
- rkcg -> rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
5420
-
5421
5467
subscription_version = rd_kafka_cgrp_subscription_set (rkcg , rktparlist );
5422
5468
5423
5469
rd_kafka_cgrp_join (rkcg , subscription_version );
@@ -5804,16 +5850,18 @@ static void rd_kafka_cgrp_join_state_serve(rd_kafka_cgrp_t *rkcg) {
5804
5850
}
5805
5851
}
5806
5852
5853
+
5807
5854
void rd_kafka_cgrp_consumer_group_heartbeat (rd_kafka_cgrp_t * rkcg ,
5808
5855
rd_bool_t full_request ,
5809
5856
rd_bool_t send_ack ) {
5810
5857
5811
- rd_kafkap_str_t * rkcg_group_instance_id = NULL ;
5812
- rd_kafkap_str_t * rkcg_client_rack = NULL ;
5813
- int max_poll_interval_ms = -1 ;
5814
- rd_kafka_topic_partition_list_t * rkcg_subscription = NULL ;
5815
- rd_kafkap_str_t * rkcg_group_remote_assignor = NULL ;
5816
- rd_kafka_topic_partition_list_t * rkcg_group_assignment = NULL ;
5858
+ rd_kafkap_str_t * rkcg_group_instance_id = NULL ;
5859
+ rd_kafkap_str_t * rkcg_client_rack = NULL ;
5860
+ int max_poll_interval_ms = -1 ;
5861
+ rd_kafka_topic_partition_list_t * rkcg_subscription_topics = NULL ;
5862
+ rd_kafkap_str_t * rkcg_subscription_regex = NULL ;
5863
+ rd_kafkap_str_t * rkcg_group_remote_assignor = NULL ;
5864
+ rd_kafka_topic_partition_list_t * rkcg_group_assignment = NULL ;
5817
5865
int32_t member_epoch = rkcg -> rkcg_generation_id ;
5818
5866
if (member_epoch < 0 )
5819
5867
member_epoch = 0 ;
@@ -5827,7 +5875,8 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
5827
5875
rkcg_client_rack = rkcg -> rkcg_client_rack ;
5828
5876
max_poll_interval_ms =
5829
5877
rkcg -> rkcg_rk -> rk_conf .max_poll_interval_ms ;
5830
- rkcg_subscription = rkcg -> rkcg_subscription ;
5878
+ rkcg_subscription_topics = rkcg -> rkcg_subscription_topics ;
5879
+ rkcg_subscription_regex = rkcg -> rkcg_subscription_regex ;
5831
5880
rkcg_group_remote_assignor = rkcg -> rkcg_group_remote_assignor ;
5832
5881
}
5833
5882
@@ -5863,14 +5912,15 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
5863
5912
(rkcg -> rkcg_consumer_flags &
5864
5913
~RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION ) |
5865
5914
RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION ;
5866
- rkcg_subscription = rkcg -> rkcg_subscription ;
5915
+ rkcg_subscription_topics = rkcg -> rkcg_subscription_topics ;
5916
+ rkcg_subscription_regex = rkcg -> rkcg_subscription_regex ;
5867
5917
5868
5918
if (rd_kafka_is_dbg (rkcg -> rkcg_rk , CGRP )) {
5869
5919
char rkcg_new_subscription_str [512 ] = "NULL" ;
5870
5920
5871
- if (rkcg_subscription ) {
5921
+ if (rkcg -> rkcg_subscription ) {
5872
5922
rd_kafka_topic_partition_list_str (
5873
- rkcg_subscription ,
5923
+ rkcg -> rkcg_subscription ,
5874
5924
rkcg_new_subscription_str ,
5875
5925
sizeof (rkcg_new_subscription_str ), 0 );
5876
5926
}
@@ -5885,7 +5935,8 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg,
5885
5935
rd_kafka_ConsumerGroupHeartbeatRequest (
5886
5936
rkcg -> rkcg_coord , rkcg -> rkcg_group_id , rkcg -> rkcg_member_id ,
5887
5937
member_epoch , rkcg_group_instance_id , rkcg_client_rack ,
5888
- max_poll_interval_ms , rkcg_subscription , rkcg_group_remote_assignor ,
5938
+ max_poll_interval_ms , rkcg_subscription_topics ,
5939
+ rkcg_subscription_regex , rkcg_group_remote_assignor ,
5889
5940
rkcg_group_assignment , RD_KAFKA_REPLYQ (rkcg -> rkcg_ops , 0 ),
5890
5941
rd_kafka_cgrp_handle_ConsumerGroupHeartbeat , NULL );
5891
5942
}
@@ -6014,22 +6065,7 @@ rd_kafka_cgrp_consumer_subscribe(rd_kafka_cgrp_t *rkcg,
6014
6065
return RD_KAFKA_RESP_ERR__FATAL ;
6015
6066
}
6016
6067
6017
- rkcg -> rkcg_flags &= ~RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
6018
6068
if (rktparlist ) {
6019
- if (rkcg -> rkcg_subscription )
6020
- rd_kafka_topic_partition_list_destroy (
6021
- rkcg -> rkcg_subscription );
6022
-
6023
- rkcg -> rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION ;
6024
-
6025
- if (rd_kafka_topic_partition_list_regex_cnt (rktparlist ) > 0 )
6026
- rkcg -> rkcg_flags |=
6027
- RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION ;
6028
-
6029
- rkcg -> rkcg_consumer_flags |=
6030
- RD_KAFKA_CGRP_CONSUMER_F_SUBSCRIBED_ONCE |
6031
- RD_KAFKA_CGRP_CONSUMER_F_SEND_NEW_SUBSCRIPTION ;
6032
-
6033
6069
rd_kafka_cgrp_subscription_set (rkcg , rktparlist );
6034
6070
rd_kafka_cgrp_consumer_expedite_next_heartbeat (
6035
6071
rkcg , "subscription changed" );
0 commit comments