Skip to content

Commit

Permalink
Fixed ListConsumerGroupOffsets not fetching offsets for all the topic…
Browse files Browse the repository at this point in the history
…s in a group with Apache Kafka version below 2.4.0
  • Loading branch information
pranavrth committed Aug 8, 2023
1 parent 07262c4 commit 734b2e4
Show file tree
Hide file tree
Showing 4 changed files with 7 additions and 1 deletion.
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -31,3 +31,4 @@ cov-int
gdbrun*.gdb
TAGS
vcpkg_installed
.idea
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

0 comments on commit 734b2e4

Please sign in to comment.