Skip to content

Commit 1cb8009

Browse files
authored
Add destroy call when leader epoch is stale (#4429)
and partition is in state active
1 parent 49f180a commit 1cb8009

File tree

2 files changed

+3
-0
lines changed

2 files changed

+3
-0
lines changed

CHANGELOG.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,8 @@ librdkafka v2.2.1 is a maintenance release:
44

55
* 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)
66
* Fixed ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0.
7+
* Add missing destroy that leads to leaking partition structure memory when there
8+
are partition leader changes and a stale leader epoch is received (#4429).
79

810

911

src/rdkafka_topic.c

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -677,6 +677,7 @@ static int rd_kafka_toppar_leader_update(rd_kafka_topic_t *rkt,
677677
rktp->rktp_leader_epoch);
678678
if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_ACTIVE) {
679679
rd_kafka_toppar_unlock(rktp);
680+
rd_kafka_toppar_destroy(rktp); /* from get() */
680681
return 0;
681682
}
682683
}

0 commit comments

Comments
 (0)