mirror of
https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
synced 2025-11-01 19:18:04 +00:00
Merge pull request #247 from filimonov/kafka_destroy_flags3
Add support for rd_kafka_destroy_flags.
This commit is contained in:
@@ -340,6 +340,18 @@ public:
|
||||
*/
|
||||
int get_out_queue_length() const;
|
||||
|
||||
#if RD_KAFKA_VERSION >= RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION
|
||||
/**
|
||||
* \brief Sets flags for rd_kafka_destroy_flags()
|
||||
*/
|
||||
void set_destroy_flags(int destroy_flags);
|
||||
|
||||
/**
|
||||
* \brief Returns flags for rd_kafka_destroy_flags()
|
||||
*/
|
||||
int get_destroy_flags() const;
|
||||
|
||||
#endif
|
||||
/**
|
||||
* \brief Cancels the current callback dispatcher
|
||||
*
|
||||
@@ -357,7 +369,14 @@ protected:
|
||||
private:
|
||||
static const std::chrono::milliseconds DEFAULT_TIMEOUT;
|
||||
|
||||
using HandlePtr = std::unique_ptr<rd_kafka_t, decltype(&rd_kafka_destroy)>;
|
||||
struct HandleDeleter {
|
||||
explicit HandleDeleter(const KafkaHandleBase* handle_base_ptr) : handle_base_ptr_{handle_base_ptr} {}
|
||||
void operator()(rd_kafka_t* handle);
|
||||
private:
|
||||
const KafkaHandleBase * handle_base_ptr_;
|
||||
};
|
||||
|
||||
using HandlePtr = std::unique_ptr<rd_kafka_t, HandleDeleter>;
|
||||
using TopicConfigurationMap = std::unordered_map<std::string, TopicConfiguration>;
|
||||
|
||||
Topic get_topic(const std::string& name, rd_kafka_topic_conf_t* conf);
|
||||
@@ -373,6 +392,7 @@ private:
|
||||
TopicConfigurationMap topic_configurations_;
|
||||
std::mutex topic_configurations_mutex_;
|
||||
HandlePtr handle_;
|
||||
int destroy_flags_;
|
||||
};
|
||||
|
||||
} // cppkafka
|
||||
|
||||
@@ -51,5 +51,6 @@
|
||||
#define RD_KAFKA_EVENT_STATS_SUPPORT_VERSION 0x000b0000 //v0.11.0.00
|
||||
#define RD_KAFKA_MESSAGE_STATUS_SUPPORT_VERSION 0x01000002 //v1.0.0.02
|
||||
#define RD_KAFKA_STORE_OFFSETS_SUPPORT_VERSION 0x00090501 //v0.9.5.01
|
||||
#define RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION 0x000b0600 //v0.11.6
|
||||
|
||||
#endif // CPPKAFKA_MACROS_H
|
||||
|
||||
@@ -48,7 +48,7 @@ namespace cppkafka {
|
||||
const milliseconds KafkaHandleBase::DEFAULT_TIMEOUT{1000};
|
||||
|
||||
KafkaHandleBase::KafkaHandleBase(Configuration config)
|
||||
: timeout_ms_(DEFAULT_TIMEOUT), config_(move(config)), handle_(nullptr, nullptr) {
|
||||
: timeout_ms_(DEFAULT_TIMEOUT), config_(move(config)), handle_(nullptr, HandleDeleter(this)), destroy_flags_(0) {
|
||||
auto& maybe_config = config_.get_default_topic_configuration();
|
||||
if (maybe_config) {
|
||||
maybe_config->set_as_opaque();
|
||||
@@ -213,7 +213,7 @@ void KafkaHandleBase::yield() const {
|
||||
}
|
||||
|
||||
void KafkaHandleBase::set_handle(rd_kafka_t* handle) {
|
||||
handle_ = HandlePtr(handle, &rd_kafka_destroy);
|
||||
handle_ = HandlePtr(handle, HandleDeleter(this));
|
||||
}
|
||||
|
||||
Topic KafkaHandleBase::get_topic(const string& name, rd_kafka_topic_conf_t* conf) {
|
||||
@@ -285,4 +285,25 @@ rd_kafka_conf_t* KafkaHandleBase::get_configuration_handle() {
|
||||
return config_.get_handle();
|
||||
}
|
||||
|
||||
#if RD_KAFKA_VERSION >= RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION
|
||||
|
||||
void KafkaHandleBase::set_destroy_flags(int destroy_flags) {
|
||||
destroy_flags_ = destroy_flags;
|
||||
};
|
||||
|
||||
int KafkaHandleBase::get_destroy_flags() const {
|
||||
return destroy_flags_;
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
|
||||
void KafkaHandleBase::HandleDeleter::operator()(rd_kafka_t* handle) {
|
||||
#if RD_KAFKA_VERSION >= RD_KAFKA_DESTROY_FLAGS_SUPPORT_VERSION
|
||||
rd_kafka_destroy_flags(handle, handle_base_ptr_->get_destroy_flags());
|
||||
#else
|
||||
rd_kafka_destroy(handle);
|
||||
#endif
|
||||
}
|
||||
|
||||
} // cppkafka
|
||||
|
||||
Reference in New Issue
Block a user