diff --git a/CHANGELOG.md b/CHANGELOG.md index 0fcf87c993..8e29fcebe6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,14 @@ +# librdkafka v1.6.0 + +## Fixes + +### General fixes + + * `rd_kafka_topic_opaque()` (used by the C++ API) would cause object + refcounting issues when used on light-weight (error-only) topic objects + such as consumer errors (#2693) + + # librdkafka v1.5.0 The v1.5.0 release brings usability improvements, enhancements and fixes to diff --git a/src/rdkafka_topic.c b/src/rdkafka_topic.c index d4ea0bbaaf..f6eb77bedc 100644 --- a/src/rdkafka_topic.c +++ b/src/rdkafka_topic.c @@ -119,7 +119,9 @@ void rd_kafka_topic_destroy_final (rd_kafka_topic_t *rkt) { } /** - * Application destroy + * @brief Application topic object destroy. + * @warning MUST ONLY BE CALLED BY THE APPLICATION. + * Use rd_kafka_topic_destroy0() for all internal use. */ void rd_kafka_topic_destroy (rd_kafka_topic_t *app_rkt) { rd_kafka_lwtopic_t *lrkt; @@ -1487,7 +1489,7 @@ void *rd_kafka_topic_opaque (const rd_kafka_topic_t *app_rkt) { opaque = rkt->rkt_conf.opaque; - rd_kafka_topic_destroy(rkt); /* loose refcnt from find() */ + rd_kafka_topic_destroy0(rkt); /* loose refcnt from find() */ return opaque; }