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

Fix for a Fetch regression when connecting to Apache Kafka < 2.7 #4871

Merged
merged 2 commits into from
Oct 16, 2024
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
18 changes: 18 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,21 @@
# librdkafka v2.6.1

librdkafka v2.6.1 is a maintenance release:

* Fix for a Fetch regression when connecting to Apache Kafka < 2.7 (#4871).


## Fixes

### Consumer fixes

* Issues: #4870
Fix for a Fetch regression when connecting to Apache Kafka < 2.7, causing
fetches to fail.
Happening since v2.6.0 (#4871)



# librdkafka v2.6.0

librdkafka v2.6.0 is a feature release:
Expand Down
4 changes: 2 additions & 2 deletions src/rdkafka_fetcher.c
Original file line number Diff line number Diff line change
Expand Up @@ -996,8 +996,8 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) {

/* Fallback to version 12 if topic id is null which can happen if
* inter.broker.protocol.version is < 2.8 */
ApiVersion =
ApiVersion > 12 && can_use_topic_ids(rkb) ? ApiVersion : 12;
if (ApiVersion > 12 && !can_use_topic_ids(rkb))
ApiVersion = 12;

rkbuf = rd_kafka_buf_new_flexver_request(
rkb, RD_KAFKAP_Fetch, 1,
Expand Down