Skip to content
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

Fixed ListConsumerGroupOffsets not fetching offsets for all the topic s in a group with Apache Kafka version below 2.4.0. #4346

Merged
merged 2 commits into from
Aug 14, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
librdkafka v2.2.1 is a maintenance release:

* Added Topic id to the metadata response which is part of the [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers)
* Fixed ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0.



Expand Down
2 changes: 1 addition & 1 deletion src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -1180,7 +1180,7 @@ void rd_kafka_OffsetFetchRequest(rd_kafka_broker_t *rkb,
rkbuf, parts, rd_false /*include invalid offsets*/,
rd_false /*skip valid offsets */, fields);
} else {
rd_kafka_buf_write_arraycnt_pos(rkbuf);
rd_kafka_buf_write_arraycnt(rkbuf, PartCnt);
}

if (ApiVersion >= 7) {
Expand Down
4 changes: 4 additions & 0 deletions tests/0081-admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -4308,7 +4308,9 @@ static void do_test_apis(rd_kafka_type_t cltype) {
do_test_DeleteConsumerGroupOffsets(
"main queue", rk, mainq, 1500,
rd_true /*with subscribing consumer*/);
}

if (test_broker_version >= TEST_BRKVER(2, 5, 0, 0)) {
/* Alter committed offsets */
do_test_AlterConsumerGroupOffsets("temp queue", rk, NULL, -1,
rd_false, rd_true);
Expand All @@ -4321,7 +4323,9 @@ static void do_test_apis(rd_kafka_type_t cltype) {
"main queue", rk, mainq, 1500,
rd_true, /*with subscribing consumer*/
rd_true);
}

if (test_broker_version >= TEST_BRKVER(2, 0, 0, 0)) {
/* List committed offsets */
do_test_ListConsumerGroupOffsets("temp queue", rk, NULL, -1,
rd_false, rd_false);
Expand Down