From c120b4c875666969d6757f192810f6f8f100b4a5 Mon Sep 17 00:00:00 2001 From: Magnus Edenhill Date: Thu, 20 Aug 2020 15:15:39 +0200 Subject: [PATCH] Use correct internal topic object destructor to avoid C++ API crash (#2963) .. for consumer errors that originate from light-weight topic objects. --- src/rdkafka_topic.c | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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; }