-
Notifications
You must be signed in to change notification settings - Fork 3.2k
KIP 848: Added support for DescribeConsumerGroup for consumer protocol groups #4941
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
🎉 All Contributor License Agreements have been signed. Ready to merge. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
First pass comments.
src/rdkafka.h
Outdated
/** | ||
* @brief Gets target assignment of \p member. | ||
* | ||
* @param member The group member. | ||
* | ||
* @return The target assignment. | ||
* | ||
* @remark The lifetime of the returned memory is the same | ||
* as the lifetime of the \p member object. | ||
*/ | ||
RD_EXPORT | ||
const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment( | ||
const rd_kafka_MemberDescription_t *member); | ||
|
||
/** | ||
* @brief Gets target assigned partitions of a member \p assignment. | ||
* | ||
* @param assignment The group member assignment. | ||
* | ||
* @return The target assigned partitions. | ||
* | ||
* @remark The lifetime of the returned memory is the same | ||
* as the lifetime of the \p assignment object. | ||
*/ | ||
RD_EXPORT | ||
const rd_kafka_topic_partition_list_t * | ||
rd_kafka_MemberAssignment_target_partitions( | ||
const rd_kafka_MemberAssignment_t *assignment); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What are we doing for the old protocol on these new functions?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is giving NULL as default value in these cases. I have updated it in the comments
CHANGELOG.md
Outdated
@@ -2,6 +2,7 @@ | |||
|
|||
librdkafka v2.8.0 is a maintenance release: | |||
|
|||
* DescribeConsumerGroup now supports new consumer protocol groups (#4922). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add that new fields type and target assignment is added. Target assignment is only applicable for Consumer
protocol. Explain about the default values as well.
examples/describe_consumer_groups.c
Outdated
const rd_kafka_topic_partition_list_t *target_topic_partitions = | ||
rd_kafka_MemberAssignment_partitions(target_assignment); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will give NULL pointer exception as target_assignment
is only applicable for consumer
protocol. It should be NULL for classic
protocol and default value should be NULL.
examples/describe_consumer_groups.c
Outdated
if (!target_topic_partitions) { | ||
printf(" No target assignment\n"); | ||
} else if (target_topic_partitions->cnt == 0) { | ||
printf(" Empty target assignment\n"); | ||
} else { | ||
printf(" Target assignment:\n"); | ||
print_partition_list(stdout, target_topic_partitions, 0, | ||
" "); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is different for classic
protocol.
src/rdkafka_proto.h
Outdated
[RD_KAFKAP_ListTransactions] = "ListTransactions", | ||
[RD_KAFKAP_AllocateProducerIds] = "AllocateProducerIds", | ||
[RD_KAFKAP_ConsumerGroupHeartbeat] = "ConsumerGroupHeartbeat", | ||
[RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Better add this at the end.
src/rdkafka_request.c
Outdated
*/ | ||
rd_kafka_resp_err_t | ||
rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb, | ||
const rd_list_t *groups /*(char*)*/, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be char **
and accept count as well. Check older API.
src/rdkafka_request.c
Outdated
rd_kafka_resp_cb_t *resp_cb, | ||
void *opaque) { | ||
rd_kafka_buf_t *rkbuf; | ||
int16_t maxApiVersion = 0; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No need for this variable. Used only once.
src/rdkafka_request.c
Outdated
int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported( | ||
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, maxApiVersion, NULL); | ||
size_t ofGroupsArrayCnt; | ||
int grp_ids_cnt = rd_list_cnt(groups); | ||
int i, include_authorized_operations; | ||
char *group; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported( | |
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, maxApiVersion, NULL); | |
size_t ofGroupsArrayCnt; | |
int grp_ids_cnt = rd_list_cnt(groups); | |
int i, include_authorized_operations; | |
char *group; | |
size_t ofGroupsArrayCnt; | |
int grp_ids_cnt = rd_list_cnt(groups); | |
int i, include_authorized_operations; | |
char *group; | |
int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported( | |
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, maxApiVersion, NULL); |
src/rdkafka_request.c
Outdated
rkbuf = rd_kafka_buf_new_flexver_request( | ||
rkb, RD_KAFKAP_ConsumerGroupDescribe, 1, | ||
4 /* rd_kafka_buf_write_arraycnt_pos */ + | ||
1 /* IncludeAuthorizedOperations */ + 1 /* tags */ + | ||
32 * grp_ids_cnt /* Groups */, | ||
rd_true /* flexver */); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
alignment
src/rdkafka_admin.c
Outdated
if (newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && | ||
newgroupres->error && | ||
(newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | ||
newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check for memory leak in this condition as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Still reviewing. Few comments. Check for possible SegFaults.
src/rdkafka_admin.c
Outdated
src->assignment.partitions); | ||
return rd_kafka_MemberDescription_new( | ||
src->client_id, src->consumer_id, src->group_instance_id, src->host, | ||
src->assignment.partitions, src->target_assignment.partitions); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will give a SegFault if target assignment is NULL
src/rdkafka_admin.c
Outdated
if (member->target_assignment.partitions) | ||
rd_kafka_topic_partition_list_destroy( | ||
member->target_assignment.partitions); | ||
rd_free(member); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same changes of SegFault.
src/rdkafka_admin.c
Outdated
RD_LIST_FOREACH(group, groups, i) { | ||
groups_arr[i] = rd_list_elem(groups, i); | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No need for this. You can directly use groups->rl_elems
2b0883d
to
b09e721
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Comments on Implementation part.
CHANGELOG.md
Outdated
@@ -160,6 +160,7 @@ librdkafka v2.10.0 is a feature release: | |||
|
|||
librdkafka v2.8.0 is a maintenance release: | |||
|
|||
* DescribeConsumerGroup now supports new consumer protocol groups. Two field type and target assignment has also been added. Target Assignment is only valid for the new Consumer protocol and its default value is NULL. (#4922). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Move changelog to 2.10.0. Add [KIP-848] tag.
CHANGELOG.md
Outdated
@@ -160,6 +160,7 @@ librdkafka v2.10.0 is a feature release: | |||
|
|||
librdkafka v2.8.0 is a maintenance release: | |||
|
|||
* DescribeConsumerGroup now supports new consumer protocol groups. Two field type and target assignment has also been added. Target Assignment is only valid for the new Consumer protocol and its default value is NULL. (#4922). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* DescribeConsumerGroup now supports new consumer protocol groups. Two field type and target assignment has also been added. Target Assignment is only valid for the new Consumer protocol and its default value is NULL. (#4922). | |
* `rd_kafka_DescribeConsumerGroups()` now supports KIP-848 introduced `consumer` groups. Two new fields for consumer group type and target assignment has also been added. Type defines whether this group is a `classic` or `consumer` group. Target assignment is only valid for the `consumer` protocol and its defaults to NULL. (#4922). |
src/rdkafka_admin.c
Outdated
const rd_kafka_topic_partition_list_t * | ||
rd_kafka_MemberAssignment_target_partitions( | ||
const rd_kafka_MemberAssignment_t *assignment) { | ||
if (assignment == NULL) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if (assignment == NULL) | |
if (!assignment) |
src/rdkafka.h
Outdated
* | ||
* @param member The group member. | ||
* | ||
* @return The target assignment. It returns NULL for CLASSIC GROUPS. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* @return The target assignment. It returns NULL for CLASSIC GROUPS. | |
* @return The target assignment for `consumer` group types. It returns NULL for the `classic` group types. |
src/rdkafka.h
Outdated
RD_EXPORT | ||
const rd_kafka_topic_partition_list_t * | ||
rd_kafka_MemberAssignment_target_partitions( | ||
const rd_kafka_MemberAssignment_t *assignment); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can update rd_kafka_MemberDescription_assignment
itself. If assignment is NULL there then it will raise a SegFault there. We can update that function instead of creating a new one.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let it be as it is as this is getter function.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should remove the new function. there is no field like target_partitions
in MemberAssignment. We should modify rd_kafka_MemberAssignment_partitions
like earlier.
(newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | ||
newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { | ||
rko_fanout->rko_u.admin_request.fanout.outstanding++; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Explain this in comments.
src/rdkafka_admin.c
Outdated
if (newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && | ||
newgroupres->error && | ||
(newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | ||
newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Explain what we are doing here in comments.
src/rdkafka_op.h
Outdated
RD_KAFKA_OP_ELECTLEADERS, /**< Admin: | ||
* ElectLeaders | ||
* u.admin_request */ | ||
RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, /**< Admin: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this is not needed altogether. Try using the old one only.
src/rdkafka_request.c
Outdated
ofGroupsArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); | ||
rd_kafka_buf_finalize_arraycnt(rkbuf, ofGroupsArrayCnt, group_cnt); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ofGroupsArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); | |
rd_kafka_buf_finalize_arraycnt(rkbuf, ofGroupsArrayCnt, group_cnt); | |
rd_kafka_buf_write_arraycnt(rkbuf, group_cnt); |
src/rdkafka_request.c
Outdated
while (group_cnt-- > 0) { | ||
rd_kafka_buf_write_str(rkbuf, groups[group_cnt], -1); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Write in same order as request.
7075a8c
to
ee0e92f
Compare
src/rdkafka_admin.c
Outdated
char *group_id = NULL, *group_state = NULL, *assignor_name = NULL, | ||
*error_str = NULL, *host = NULL; | ||
char *member_id = NULL, *instance_id = NULL, *client_id = NULL, | ||
*client_host = NULL; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Merge these two
src/rdkafka_admin.c
Outdated
member_id = RD_KAFKAP_STR_DUP(&MemberId); | ||
instance_id = RD_KAFKAP_STR_DUP(&InstanceId); | ||
client_id = RD_KAFKAP_STR_DUP(&ClientId); | ||
client_host = RD_KAFKAP_STR_DUP(&ClientHost); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is leaking. Free them after using and assign them as NULL.
src/rdkafka_admin.c
Outdated
rd_assert(rko_partial->rko_evtype == | ||
RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || | ||
rko_partial->rko_evtype == | ||
RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can have only 1 of these. Keep the older one.
src/rdkafka.h
Outdated
RD_EXPORT | ||
const rd_kafka_topic_partition_list_t * | ||
rd_kafka_MemberAssignment_target_partitions( | ||
const rd_kafka_MemberAssignment_t *assignment); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should remove the new function. there is no field like target_partitions
in MemberAssignment. We should modify rd_kafka_MemberAssignment_partitions
like earlier.
55b2fe9
to
5b7ed97
Compare
src/rdkafka.h
Outdated
* @param member The group member. | ||
* | ||
* @return The target assignment for `consumer` group types. | ||
* It returns NULL for the `classic` group types. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* It returns NULL for the `classic` group types. | |
* Returns NULL for the `classic` group types. |
src/rdkafka_admin.c
Outdated
if (member->target_assignment->partitions) | ||
rd_kafka_topic_partition_list_destroy( | ||
member->target_assignment->partitions); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use RD_IF_FREE for this and other if conditions in the function.
src/rdkafka_admin.c
Outdated
if (assignment) | ||
return assignment->partitions; | ||
else | ||
return NULL; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if (assignment) | |
return assignment->partitions; | |
else | |
return NULL; | |
if (!assignment) | |
return NULL; | |
return assignment->partitions; |
src/rdkafka.h
Outdated
* @remark The lifetime of the returned memory is the same | ||
* as the lifetime of the \p grpdesc object. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Given it's not returning a pointer this isn't necessary
* @remark The lifetime of the returned memory is the same | |
* as the lifetime of the \p grpdesc object. |
|
||
nodeid = rkb->rkb_nodeid; | ||
rd_kafka_broker_lock(rkb); | ||
host = rd_strdup(rkb->rkb_origname); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Free host. Memory leak there.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Free node as well.
src/rdkafka_admin.c
Outdated
if (assignment) | ||
rd_kafka_topic_partition_list_destroy( | ||
assignment); | ||
if (target_assignment) | ||
rd_kafka_topic_partition_list_destroy( | ||
target_assignment); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
RD_IF_FREE
src/rdkafka_admin.c
Outdated
rd_kafka_MemberDescription_t *member = NULL; | ||
rd_kafka_topic_partition_list_t *assignment = NULL, | ||
*target_assignment = | ||
NULL; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These will leak in err_parse
case. Free them properly.
src/rdkafka_admin.c
Outdated
identify it for further processing with the old protocol and | ||
eventually in rd_kafka_DescribeConsumerGroupsResponse_parse | ||
we will set the ConsumerGroupType to Unknown */ | ||
if (error == NULL) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if (error == NULL) { | |
if (!error) { |
src/rdkafka_admin.c
Outdated
if ((newgroupres->error && | ||
newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE) || | ||
(newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && | ||
newgroupres->error && | ||
(newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | ||
newgroupres->error->code == | ||
RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION))) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Refactor this condition to a new function. Something like do_fallback_to_old_consumer_group_descibe()
src/rdkafka_admin.c
Outdated
static const struct rd_kafka_admin_worker_cbs cbs = { | ||
rd_kafka_admin_DescribeConsumerGroupsRequest, | ||
rd_kafka_DescribeConsumerGroupsResponse_parse, | ||
}; | ||
rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( | ||
rko_fanout->rko_rk, RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, | ||
RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, &cbs, | ||
&rko_fanout->rko_u.admin_request.options, | ||
rko_fanout->rko_rk->rk_ops); | ||
|
||
rko->rko_u.admin_request.fanout_parent = rko_fanout; | ||
rko->rko_u.admin_request.broker_id = | ||
RD_KAFKA_ADMIN_TARGET_COORDINATOR; | ||
rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP; | ||
rko->rko_u.admin_request.coordkey = rd_strdup(grp); | ||
|
||
/* Set the group name as the opaque so the fanout worker use it | ||
* to fill in errors. | ||
* References rko_fanout's memory, which will always outlive | ||
* the fanned out op. */ | ||
rd_kafka_AdminOptions_set_opaque( | ||
&rko->rko_u.admin_request.options, (void *)grp); | ||
|
||
rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free); | ||
rd_list_add(&rko->rko_u.admin_request.args, rd_strdup(grp)); | ||
|
||
rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Extract this into a method as well. We can use this method below as well.
@@ -8453,8 +8786,8 @@ void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk, | |||
* coordinator into one op. */ | |||
for (i = 0; i < groups_cnt; i++) { | |||
static const struct rd_kafka_admin_worker_cbs cbs = { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can use earlier created method here as well.
src/rdkafka_admin.c
Outdated
@@ -8296,9 +8323,11 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, | |||
client_id = RD_KAFKAP_STR_DUP(&ClientId); | |||
client_host = RD_KAFKAP_STR_DUP(&ClientHost); | |||
|
|||
/* Target Assignment is NULL for the classic protocol */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
/* Target Assignment is NULL for the classic protocol */ | |
/* Target Assignment is `NULL` for the `classic` protocol as there is no concept of Target Assignment there. */ |
src/rdkafka_admin.h
Outdated
@@ -527,6 +527,8 @@ struct rd_kafka_MemberDescription_s { | |||
char *group_instance_id; /**< Group instance id */ | |||
char *host; /**< Group member host */ | |||
rd_kafka_MemberAssignment_t assignment; /**< Member assignment */ | |||
rd_kafka_MemberAssignment_t | |||
*target_assignment; /**< Target assignment */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
*target_assignment; /**< Target assignment */ | |
*target_assignment; /**< Target assignment. `NULL` for `classic` protocol */ |
tests/0081-admin.c
Outdated
if (i < 2) { | ||
/* Classic Protocol */ | ||
test_conf_init(&conf, NULL, 0); | ||
test_conf_set(conf, "group.protocol", "classic"); | ||
} else { | ||
/* Consumer Protocol */ | ||
test_conf_init(&conf, NULL, 0); | ||
test_conf_set(conf, "group.protocol", "consumer"); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use protocols
directly instead of this if-else.
if (i < 2) { | |
/* Classic Protocol */ | |
test_conf_init(&conf, NULL, 0); | |
test_conf_set(conf, "group.protocol", "classic"); | |
} else { | |
/* Consumer Protocol */ | |
test_conf_init(&conf, NULL, 0); | |
test_conf_set(conf, "group.protocol", "consumer"); | |
} | |
test_conf_init(&conf, NULL, 0); | |
test_conf_set(conf, "group.protocol", protocols[i]); |
tests/0081-admin.c
Outdated
"client_id_%" PRId32, i); | ||
|
||
test_conf_set(conf, "client.id", client_ids[i]); | ||
test_conf_set(conf, "session.timeout.ms", "5000"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This property is not needed for this test.
tests/0081-admin.c
Outdated
while (1) { | ||
rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); | ||
TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", | ||
rd_kafka_event_name(rkev), | ||
TIMING_DURATION(&timing) / 1000.0f); | ||
if (rkev == NULL) | ||
continue; | ||
if (rd_kafka_event_error(rkev)) | ||
TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), | ||
rd_kafka_event_error_string(rkev)); | ||
|
||
if (rd_kafka_event_type(rkev) == | ||
RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { | ||
break; | ||
} | ||
|
||
rd_kafka_event_destroy(rkev); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can extract this functionality into a method in the same file or in test.c
file which can be used by other functions in this test file.
tests/0081-admin.c
Outdated
expected_DescribeConsumerGroups_result_t *exp = &expected[i]; | ||
rd_kafka_resp_err_t exp_err = exp->err; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use expected[i] directly.
tests/0081-admin.c
Outdated
|
||
if (strcmp(protocols[i], "classic") == 0) { | ||
TEST_ASSERT( | ||
state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE || | ||
state == RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY, | ||
"Expected Stable or Empty state for classic " | ||
"protocol, got %s.", | ||
rd_kafka_consumer_group_state_name(state)); | ||
} else if (strcmp(protocols[i], "consumer") == 0) { | ||
TEST_ASSERT(state == | ||
RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, | ||
"Expected Stable state, got %s.", | ||
rd_kafka_consumer_group_state_name(state)); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why are we not checking the type here?
|
||
nodeid = rkb->rkb_nodeid; | ||
rd_kafka_broker_lock(rkb); | ||
host = rd_strdup(rkb->rkb_origname); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Free node as well.
tests/0081-admin.c
Outdated
} else { | ||
do_test_DescribeConsumerGroups_Compatibility("temp queue", rk, | ||
NULL, -1); | ||
do_test_DescribeConsumerGroups_Compatibility("main queue", rk, | ||
mainq, 1500); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These all test can be tested in all the scenarios starting Broker version 4.0. Including if condition as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You will need to use IncrementalAlterConfigs to change the session timeout and heartbeat timeout for consumer protocol case though for the consumer to leave the group and delete the group.
tests/0081-admin.c
Outdated
} else { | ||
/* Consumer Protocol */ | ||
test_conf_init(&conf, NULL, 0); | ||
test_conf_set(conf, "group.protocol", "consumer"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is overriden in the test_create_consumer()
function. This test would fail in that case if we test for the type as well.
src/rdkafka_admin.c
Outdated
rd_kafka_buf_skip_tags(reply); | ||
|
||
member_id = RD_KAFKAP_STR_DUP(&MemberId); | ||
instance_id = RD_KAFKAP_STR_DUP(&InstanceId); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This can be NULL. It will be a Segmentation fault in this case. Fix this.
src/rdkafka_admin.c
Outdated
RD_IF_FREE(host, rd_free); | ||
RD_IF_FREE(node, rd_kafka_Node_destroy); | ||
rd_kafka_buf_skip_tags(reply); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
RD_IF_FREE(host, rd_free); | |
RD_IF_FREE(node, rd_kafka_Node_destroy); | |
rd_kafka_buf_skip_tags(reply); | |
rd_kafka_buf_skip_tags(reply); | |
RD_IF_FREE(host, rd_free); | |
RD_IF_FREE(node, rd_kafka_Node_destroy); |
src/rdkafka_admin.c
Outdated
groupres->error->code == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE) || | ||
(groupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER
for UNSUPPORTED_FEATURE
error as well. If we get this error (which might happen in old brokers), we will keep on looping.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks Pratyush, these are my comments about the PR
CHANGELOG.md
Outdated
@@ -39,6 +39,8 @@ librdkafka v2.10.0 is a feature release: | |||
leader change and offset validation (#4970). | |||
* Fix the Nagle algorithm (TCP_NODELAY) on broker sockets to not be enabled | |||
by default (#4986). | |||
* [KIP-848] `rd_kafka_DescribeConsumerGroups()` now supports KIP-848 introduced `consumer` groups. Two new fields for consumer group type and target assignment has also been added. Type defines whether this group is a `classic` or `consumer` group. Target assignment is only valid for the `consumer` protocol and its defaults to NULL. (#4922). | |||
* [KIP-848] Group Config is now supported in IncrementalAlterConfigs and DescribeConfigs. (#4939). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* [KIP-848] Group Config is now supported in IncrementalAlterConfigs and DescribeConfigs. (#4939). | |
* [KIP-848] Group Config is now supported in AlterConfigs, IncrementalAlterConfigs and DescribeConfigs. (#4939). |
CHANGELOG.md
Outdated
@@ -39,6 +39,8 @@ librdkafka v2.10.0 is a feature release: | |||
leader change and offset validation (#4970). | |||
* Fix the Nagle algorithm (TCP_NODELAY) on broker sockets to not be enabled | |||
by default (#4986). | |||
* [KIP-848] `rd_kafka_DescribeConsumerGroups()` now supports KIP-848 introduced `consumer` groups. Two new fields for consumer group type and target assignment has also been added. Type defines whether this group is a `classic` or `consumer` group. Target assignment is only valid for the `consumer` protocol and its defaults to NULL. (#4922). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* [KIP-848] `rd_kafka_DescribeConsumerGroups()` now supports KIP-848 introduced `consumer` groups. Two new fields for consumer group type and target assignment has also been added. Type defines whether this group is a `classic` or `consumer` group. Target assignment is only valid for the `consumer` protocol and its defaults to NULL. (#4922). | |
* [KIP-848] `rd_kafka_DescribeConsumerGroups()` now supports KIP-848 introduced `consumer` groups. Two new fields for consumer group type and target assignment have also been added. Type defines whether this group is a `classic` or `consumer` group. Target assignment is only valid for the `consumer` protocol and its defaults to NULL. (#4922). |
examples/describe_consumer_groups.c
Outdated
const rd_kafka_MemberAssignment_t *target_assignment = | ||
rd_kafka_MemberDescription_target_assignment(member); | ||
const rd_kafka_topic_partition_list_t *target_topic_partitions = | ||
rd_kafka_MemberAssignment_partitions(target_assignment); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This function shouldn't check the argument is NULL, instead it mustn't be called with a NULL argument.
rd_kafka_MemberAssignment_partitions(target_assignment); | |
target_assignment | |
? rd_kafka_MemberAssignment_partitions(target_assignment) | |
: NULL; |
examples/describe_consumer_groups.c
Outdated
@@ -212,9 +228,10 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { | |||
rd_kafka_Node_port(coordinator)); | |||
} | |||
printf( | |||
"Group \"%s\", partition assignor \"%s\", " | |||
"Group \"%s\", partition assignor \"%s\", type \"%s\" " | |||
" state %s%s, with %" PRId32 " member(s)\n", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As with other field let's add the quotes
" state %s%s, with %" PRId32 " member(s)\n", | |
" state \"%s\"%s, with %" PRId32 " member(s)\n", |
src/rdkafka_admin.c
Outdated
if (!assignment) | ||
return NULL; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same, here it shouldn't check for NULL
size_t i; | ||
|
||
int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported( | ||
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, 0, NULL); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We need to create the local _UNSUPPORTED_FEATURE
error after getting the ApiVersion
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, 0, NULL); | |
rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, 0, NULL); | |
if (ApiVersion == -1) { | |
return rd_kafka_error_new( | |
RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE, | |
"ConsumerGroupDescribe (KIP-848) " | |
"not supported by broker, " | |
"requires broker version >= 4.0.0"); | |
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This returns the error on the request part itself and then the admin worker fails. We removed it during the review. Check -
librdkafka/src/rdkafka_admin.c
Lines 8176 to 8186 in 85b9cb4
error = rd_kafka_ConsumerGroupDescribeRequest( | |
rkb, (char **)groups->rl_elems, groups_cnt, | |
include_authorized_operations, replyq, resp_cb, opaque); | |
if (error) { | |
rd_snprintf(errstr, errstr_size, "%s", | |
rd_kafka_error_string(error)); | |
err = rd_kafka_error_code(error); | |
rd_kafka_error_destroy(error); | |
return err; | |
} |
We are getting same error in the response merge function if we remove it from here through rd_kafka_send
I think.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In case returning this error as in other requests, the duplicate
rd_kafka_enq_once_del_source(eonce, "coordinator response");
on rd_kafka_admin_coord_request
should be removed otherwise it the eonce id destroyed before calling the callback.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We are getting same error in the response merge function if we remove it from here through rd_kafka_send I think.
Yes the error is returned by rd_kafka_send
but it's not correct to continue with request construction if the ApiVersion
is -1
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Changed it to not fail when the admin result when request is rd_kafka_admin_ConsumerGroupDescribeRequest
and the err returned is Unsupported feature
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't know if this is correct to hardcode a condition for this specific case in general rd_kafka_admin_coord_request
function. Let's understand this offline.
85b9cb4
to
39f225d
Compare
…kafka_admin_ConsumerGroupDescribeRequest
src/rdkafka_admin.c
Outdated
UNSUPPORTED_VERSION(35) we need to send a request to the old | ||
protocol. | ||
*/ | ||
static rd_bool_t rd_kafka_admin_describe_consumer_group_fallback_to_classic( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
static rd_bool_t rd_kafka_admin_describe_consumer_group_fallback_to_classic( | |
static rd_bool_t rd_kafka_admin_describe_consumer_group_do_fallback_to_classic( |
src/rdkafka_admin.c
Outdated
if (groupres->error && | ||
(groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | ||
groupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION || | ||
groupres->error->code == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE)) { | ||
return rd_true; | ||
} | ||
return rd_false; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if (groupres->error && | |
(groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | |
groupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION || | |
groupres->error->code == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE)) { | |
return rd_true; | |
} | |
return rd_false; | |
return groupres->error && | |
(groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || | |
groupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION || | |
groupres->error->code == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks Prayush and Pranav! Approving
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM!. Thanks Pratyush and Emanuele.
This PR intends to add support for calling DescribeConsumerGroups for groups created with the new consumer protocol and add backward compatibility to describe group created with the old Classic Protocol.
The working of the API remains similar. Users will continue to call the existing DescribeConsumerGroups api but internally we will be calling the new ConsumerGroupDescribe api to fetch the results. If we get GRP_NOT_FOUND or UNSUPPORTED_VERSION error, then in that case we will call the old DescribeConsumerGroup api.