Skip to content

Commit

Permalink
Add destroy call when leader epoch is stale (#4429)
Browse files Browse the repository at this point in the history
and partition is in state active
  • Loading branch information
emasab authored Sep 21, 2023
1 parent 49f180a commit 1cb8009
Show file tree
Hide file tree
Showing 2 changed files with 3 additions and 0 deletions.
2 changes: 2 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@ 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.
* Add missing destroy that leads to leaking partition structure memory when there
are partition leader changes and a stale leader epoch is received (#4429).



Expand Down
1 change: 1 addition & 0 deletions src/rdkafka_topic.c
Original file line number Diff line number Diff line change
Expand Up @@ -677,6 +677,7 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt,
rktp->rktp_leader_epoch);
if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE) {
rd_kafka_toppar_unlock(rktp);
rd_kafka_toppar_destroy(rktp); /* from get() */
return 0;
}
}
Expand Down

0 comments on commit 1cb8009

Please sign in to comment.