diff --git a/CHANGELOG.md b/CHANGELOG.md index fbc67c8242..2e6714e145 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -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). diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index 4341637bc0..3b3986d436 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -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; } }