mirror of
https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
synced 2025-11-03 12:07:57 +00:00
Add consumer rebalance callbacks
This commit is contained in:
@@ -4,6 +4,7 @@
|
|||||||
#include <vector>
|
#include <vector>
|
||||||
#include <string>
|
#include <string>
|
||||||
#include <chrono>
|
#include <chrono>
|
||||||
|
#include <functional>
|
||||||
#include "kafka_handle_base.h"
|
#include "kafka_handle_base.h"
|
||||||
#include "message.h"
|
#include "message.h"
|
||||||
|
|
||||||
@@ -14,14 +15,22 @@ class TopicConfiguration;
|
|||||||
|
|
||||||
class Consumer : public KafkaHandleBase {
|
class Consumer : public KafkaHandleBase {
|
||||||
public:
|
public:
|
||||||
Consumer(const Configuration& config);
|
using AssignmentCallback = std::function<void(const TopicPartitionList&)>;
|
||||||
|
using RevocationCallback = std::function<void(const TopicPartitionList&)>;
|
||||||
|
using RebalanceErrorCallback = std::function<void(rd_kafka_resp_err_t)>;
|
||||||
|
|
||||||
|
Consumer(Configuration config);
|
||||||
|
|
||||||
void set_timeout(const std::chrono::milliseconds timeout);
|
void set_timeout(const std::chrono::milliseconds timeout);
|
||||||
|
void set_assignment_callback(AssignmentCallback callback);
|
||||||
|
void set_revocation_callback(RevocationCallback callback);
|
||||||
|
void set_rebalance_error_callback(RebalanceErrorCallback callback);
|
||||||
|
|
||||||
void subscribe(const std::vector<std::string>& topics);
|
void subscribe(const std::vector<std::string>& topics);
|
||||||
void unsubscribe();
|
void unsubscribe();
|
||||||
|
|
||||||
void assign(const TopicPartitionList& topic_partitions);
|
void assign(const TopicPartitionList& topic_partitions);
|
||||||
|
void unassign();
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
void commit(const Message& msg);
|
void commit(const Message& msg);
|
||||||
@@ -38,10 +47,17 @@ public:
|
|||||||
private:
|
private:
|
||||||
static const std::chrono::milliseconds DEFAULT_TIMEOUT;
|
static const std::chrono::milliseconds DEFAULT_TIMEOUT;
|
||||||
|
|
||||||
|
static void rebalance_proxy(rd_kafka_t *handle, rd_kafka_resp_err_t error,
|
||||||
|
rd_kafka_topic_partition_list_t *partitions, void *opaque);
|
||||||
|
|
||||||
void commit(const Message& msg, bool async);
|
void commit(const Message& msg, bool async);
|
||||||
void commit(const TopicPartitionList& topic_partitions, bool async);
|
void commit(const TopicPartitionList& topic_partitions, bool async);
|
||||||
|
void handle_rebalance(rd_kafka_resp_err_t err, const TopicPartitionList& topic_partitions);
|
||||||
|
|
||||||
std::chrono::milliseconds timeout_ms_;
|
std::chrono::milliseconds timeout_ms_;
|
||||||
|
AssignmentCallback assignment_callback_;
|
||||||
|
RevocationCallback revocation_callback_;
|
||||||
|
RebalanceErrorCallback rebalance_error_callback_;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // cppkafka
|
} // cppkafka
|
||||||
|
|||||||
@@ -12,6 +12,8 @@ class TopicPartition;
|
|||||||
|
|
||||||
class TopicPartitionList {
|
class TopicPartitionList {
|
||||||
public:
|
public:
|
||||||
|
static TopicPartitionList make_non_owning(rd_kafka_topic_partition_list_t* handle);
|
||||||
|
|
||||||
TopicPartitionList();
|
TopicPartitionList();
|
||||||
TopicPartitionList(rd_kafka_topic_partition_list_t* handle);
|
TopicPartitionList(rd_kafka_topic_partition_list_t* handle);
|
||||||
TopicPartitionList(size_t size);
|
TopicPartitionList(size_t size);
|
||||||
@@ -40,11 +42,15 @@ public:
|
|||||||
private:
|
private:
|
||||||
static const size_t DEFAULT_CONTAINER_SIZE;
|
static const size_t DEFAULT_CONTAINER_SIZE;
|
||||||
|
|
||||||
|
struct NonOwningTag { };
|
||||||
|
|
||||||
using HandlePtr = std::unique_ptr<rd_kafka_topic_partition_list_t,
|
using HandlePtr = std::unique_ptr<rd_kafka_topic_partition_list_t,
|
||||||
decltype(&rd_kafka_topic_partition_list_destroy)>;
|
decltype(&rd_kafka_topic_partition_list_destroy)>;
|
||||||
|
|
||||||
static HandlePtr make_handle(rd_kafka_topic_partition_list_t* ptr);
|
static HandlePtr make_handle(rd_kafka_topic_partition_list_t* ptr);
|
||||||
|
|
||||||
|
TopicPartitionList(rd_kafka_topic_partition_list_t* handle, NonOwningTag);
|
||||||
|
|
||||||
rd_kafka_topic_partition_t* get_topic_partition(const TopicPartition& topic_partition) const;
|
rd_kafka_topic_partition_t* get_topic_partition(const TopicPartition& topic_partition) const;
|
||||||
|
|
||||||
HandlePtr handle_;
|
HandlePtr handle_;
|
||||||
|
|||||||
@@ -12,9 +12,17 @@ namespace cppkafka {
|
|||||||
|
|
||||||
const milliseconds Consumer::DEFAULT_TIMEOUT{1000};
|
const milliseconds Consumer::DEFAULT_TIMEOUT{1000};
|
||||||
|
|
||||||
Consumer::Consumer(const Configuration& config)
|
void Consumer::rebalance_proxy(rd_kafka_t*, rd_kafka_resp_err_t error,
|
||||||
|
rd_kafka_topic_partition_list_t *partitions, void *opaque) {
|
||||||
|
TopicPartitionList list = TopicPartitionList::make_non_owning(partitions);
|
||||||
|
static_cast<Consumer*>(opaque)->handle_rebalance(error, list);
|
||||||
|
}
|
||||||
|
|
||||||
|
Consumer::Consumer(Configuration config)
|
||||||
: timeout_ms_(DEFAULT_TIMEOUT) {
|
: timeout_ms_(DEFAULT_TIMEOUT) {
|
||||||
char error_buffer[512];
|
char error_buffer[512];
|
||||||
|
// Set ourselves as the opaque pointer
|
||||||
|
rd_kafka_conf_set_opaque(config.get_handle(), this);
|
||||||
rd_kafka_t* ptr = rd_kafka_new(RD_KAFKA_CONSUMER, config.get_handle(),
|
rd_kafka_t* ptr = rd_kafka_new(RD_KAFKA_CONSUMER, config.get_handle(),
|
||||||
error_buffer, sizeof(error_buffer));
|
error_buffer, sizeof(error_buffer));
|
||||||
if (!ptr) {
|
if (!ptr) {
|
||||||
@@ -27,6 +35,18 @@ void Consumer::set_timeout(const milliseconds timeout) {
|
|||||||
timeout_ms_ = timeout;
|
timeout_ms_ = timeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Consumer::set_assignment_callback(AssignmentCallback callback) {
|
||||||
|
assignment_callback_ = move(callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
void Consumer::set_revocation_callback(RevocationCallback callback) {
|
||||||
|
revocation_callback_ = move(callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
void Consumer::set_rebalance_error_callback(RebalanceErrorCallback callback) {
|
||||||
|
rebalance_error_callback_ = move(callback);
|
||||||
|
}
|
||||||
|
|
||||||
void Consumer::subscribe(const vector<string>& topics) {
|
void Consumer::subscribe(const vector<string>& topics) {
|
||||||
TopicPartitionList list(topics.begin(), topics.end());
|
TopicPartitionList list(topics.begin(), topics.end());
|
||||||
rd_kafka_resp_err_t error = rd_kafka_subscribe(get_handle(), list.get_handle());
|
rd_kafka_resp_err_t error = rd_kafka_subscribe(get_handle(), list.get_handle());
|
||||||
@@ -45,6 +65,11 @@ void Consumer::assign(const TopicPartitionList& topic_partitions) {
|
|||||||
check_error(error);
|
check_error(error);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Consumer::unassign() {
|
||||||
|
rd_kafka_resp_err_t error = rd_kafka_assign(get_handle(), nullptr);
|
||||||
|
check_error(error);
|
||||||
|
}
|
||||||
|
|
||||||
void Consumer::close() {
|
void Consumer::close() {
|
||||||
rd_kafka_resp_err_t error = rd_kafka_consumer_close(get_handle());
|
rd_kafka_resp_err_t error = rd_kafka_consumer_close(get_handle());
|
||||||
check_error(error);
|
check_error(error);
|
||||||
@@ -117,4 +142,26 @@ void Consumer::commit(const TopicPartitionList& topic_partitions, bool async) {
|
|||||||
check_error(error);
|
check_error(error);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Consumer::handle_rebalance(rd_kafka_resp_err_t error,
|
||||||
|
const TopicPartitionList& topic_partitions) {
|
||||||
|
if (error == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) {
|
||||||
|
if (assignment_callback_) {
|
||||||
|
assignment_callback_(topic_partitions);
|
||||||
|
}
|
||||||
|
assign(topic_partitions);
|
||||||
|
}
|
||||||
|
else if (error == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS) {
|
||||||
|
if (revocation_callback_) {
|
||||||
|
revocation_callback_(topic_partitions);
|
||||||
|
}
|
||||||
|
unassign();
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
if (rebalance_error_callback_) {
|
||||||
|
rebalance_error_callback_(error);
|
||||||
|
}
|
||||||
|
unassign();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
} // cppkafka
|
} // cppkafka
|
||||||
|
|||||||
@@ -6,6 +6,15 @@ namespace cppkafka {
|
|||||||
|
|
||||||
const size_t TopicPartitionList::DEFAULT_CONTAINER_SIZE = 5;
|
const size_t TopicPartitionList::DEFAULT_CONTAINER_SIZE = 5;
|
||||||
|
|
||||||
|
void dummy_deleter(rd_kafka_topic_partition_list_t*) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
TopicPartitionList
|
||||||
|
TopicPartitionList::make_non_owning(rd_kafka_topic_partition_list_t* handle) {
|
||||||
|
return TopicPartitionList(handle, NonOwningTag());
|
||||||
|
}
|
||||||
|
|
||||||
TopicPartitionList::TopicPartitionList()
|
TopicPartitionList::TopicPartitionList()
|
||||||
: TopicPartitionList(DEFAULT_CONTAINER_SIZE) {
|
: TopicPartitionList(DEFAULT_CONTAINER_SIZE) {
|
||||||
|
|
||||||
@@ -16,6 +25,12 @@ TopicPartitionList::TopicPartitionList(rd_kafka_topic_partition_list_t* handle)
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TopicPartitionList::TopicPartitionList(rd_kafka_topic_partition_list_t* handle,
|
||||||
|
NonOwningTag)
|
||||||
|
: handle_(handle, &dummy_deleter) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
TopicPartitionList::TopicPartitionList(size_t size)
|
TopicPartitionList::TopicPartitionList(size_t size)
|
||||||
: handle_(make_handle(rd_kafka_topic_partition_list_new(size))) {
|
: handle_(make_handle(rd_kafka_topic_partition_list_new(size))) {
|
||||||
|
|
||||||
|
|||||||
Reference in New Issue
Block a user