From f543810515816e257c03f56ce735f65c02b6c27d Mon Sep 17 00:00:00 2001 From: Alex Damian Date: Mon, 28 May 2018 14:30:01 -0400 Subject: [PATCH] Allow metadata object to be non-owning (#73) --- include/cppkafka/metadata.h | 33 ++++++++++++++++++++++++++++++++- include/cppkafka/topic.h | 7 +++++++ src/metadata.cpp | 36 ++++++++++++++++++++++++++++++++++-- 3 files changed, 73 insertions(+), 3 deletions(-) diff --git a/include/cppkafka/metadata.h b/include/cppkafka/metadata.h index 6dacf80..d5310aa 100644 --- a/include/cppkafka/metadata.h +++ b/include/cppkafka/metadata.h @@ -140,7 +140,24 @@ private: */ class CPPKAFKA_API Metadata { public: - Metadata(const rd_kafka_metadata_t* ptr); + /** + * \brief Creates a Metadata object that doesn't take ownership of the handle + * + * \param handle The handle to be used + */ + static Metadata make_non_owning(const rd_kafka_metadata_t* handle); + + /** + * \brief Constructs an empty metadata object + * + * \remark Using any methods except Metadata::get_handle on an empty metadata is undefined behavior + */ + Metadata(); + + /** + * Constructor + */ + Metadata(const rd_kafka_metadata_t* handle); /** * Gets the brokers' metadata @@ -165,8 +182,22 @@ public: * \param prefix The prefix to be looked up */ std::vector get_topics_prefixed(const std::string& prefix) const; + + /** + * Indicates whether this metadata is valid (not null) + */ + explicit operator bool() const; + + /** + * Returns the rdkakfa handle + */ + const rd_kafka_metadata_t* get_handle() const; private: using HandlePtr = std::unique_ptr; + + struct NonOwningTag { }; + + Metadata(const rd_kafka_metadata_t* handle, NonOwningTag); HandlePtr handle_; }; diff --git a/include/cppkafka/topic.h b/include/cppkafka/topic.h index 3eb972d..ac5757b 100644 --- a/include/cppkafka/topic.h +++ b/include/cppkafka/topic.h @@ -82,6 +82,13 @@ public: * \param partition The partition to check */ bool is_partition_available(int partition) const; + + /** + * Indicates whether this topic is valid (not null) + */ + explicit operator bool() const { + return handle_ != nullptr; + } /** * Returns the rdkakfa handle diff --git a/src/metadata.cpp b/src/metadata.cpp index e47765d..e7faf06 100644 --- a/src/metadata.cpp +++ b/src/metadata.cpp @@ -27,6 +27,7 @@ * */ +#include #include "metadata.h" #include "error.h" @@ -110,12 +111,31 @@ uint16_t BrokerMetadata::get_port() const { // Metadata -Metadata::Metadata(const rd_kafka_metadata_t* ptr) -: handle_(ptr, &rd_kafka_metadata_destroy) { +void dummy_metadata_destroyer(const rd_kafka_metadata_t*) { + +} + +Metadata Metadata::make_non_owning(const rd_kafka_metadata_t* handle) { + return Metadata(handle, NonOwningTag{}); +} + +Metadata::Metadata() +: handle_(nullptr, nullptr) { + +} + +Metadata::Metadata(const rd_kafka_metadata_t* handle) +: handle_(handle, &rd_kafka_metadata_destroy) { + +} + +Metadata::Metadata(const rd_kafka_metadata_t* handle, NonOwningTag) +: handle_(handle, &dummy_metadata_destroyer) { } vector Metadata::get_brokers() const { + assert(handle_); vector output; for (int i = 0; i < handle_->broker_cnt; ++i) { const rd_kafka_metadata_broker_t& broker = handle_->brokers[i]; @@ -125,6 +145,7 @@ vector Metadata::get_brokers() const { } vector Metadata::get_topics() const { + assert(handle_); vector output; for (int i = 0; i < handle_->topic_cnt; ++i) { const rd_kafka_metadata_topic_t& topic = handle_->topics[i]; @@ -134,6 +155,7 @@ vector Metadata::get_topics() const { } vector Metadata::get_topics(const unordered_set& topics) const { + assert(handle_); vector output; for (int i = 0; i < handle_->topic_cnt; ++i) { const rd_kafka_metadata_topic_t& topic = handle_->topics[i]; @@ -145,6 +167,7 @@ vector Metadata::get_topics(const unordered_set& topics) } vector Metadata::get_topics_prefixed(const string& prefix) const { + assert(handle_); vector output; for (int i = 0; i < handle_->topic_cnt; ++i) { const rd_kafka_metadata_topic_t& topic = handle_->topics[i]; @@ -156,4 +179,13 @@ vector Metadata::get_topics_prefixed(const string& prefix) const return output; } + +Metadata::operator bool() const { + return handle_ != nullptr; +} + +const rd_kafka_metadata_t* Metadata::get_handle() const { + return handle_.get(); +} + } // cppkafka