mirror of
https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
synced 2025-11-01 11:07:56 +00:00
Add pause/resume partitions wrappers
This commit is contained in:
@@ -5,7 +5,6 @@
|
||||
#include <string>
|
||||
#include <chrono>
|
||||
#include "kafka_handle_base.h"
|
||||
#include "topic_partition_list.h"
|
||||
#include "message.h"
|
||||
|
||||
namespace cppkafka {
|
||||
@@ -35,14 +34,12 @@ public:
|
||||
TopicPartitionList get_subscription();
|
||||
TopicPartitionList get_assignment();
|
||||
|
||||
|
||||
Message poll();
|
||||
private:
|
||||
static const std::chrono::milliseconds DEFAULT_TIMEOUT;
|
||||
|
||||
void commit(const Message& msg, bool async);
|
||||
void commit(const TopicPartitionList& topic_partitions, bool async);
|
||||
void check_error(rd_kafka_resp_err_t error);
|
||||
|
||||
std::chrono::milliseconds timeout_ms_;
|
||||
};
|
||||
|
||||
@@ -4,6 +4,7 @@
|
||||
#include <string>
|
||||
#include <memory>
|
||||
#include <librdkafka/rdkafka.h>
|
||||
#include "topic_partition_list.h"
|
||||
|
||||
namespace cppkafka {
|
||||
|
||||
@@ -18,6 +19,9 @@ public:
|
||||
KafkaHandleBase& operator=(const KafkaHandleBase&) = delete;
|
||||
KafkaHandleBase& operator=(KafkaHandleBase&&) = delete;
|
||||
|
||||
void pause_partitions(const TopicPartitionList& topic_partitions);
|
||||
void resume_partitions(const TopicPartitionList& topic_partitions);
|
||||
|
||||
rd_kafka_t* get_handle();
|
||||
Topic get_topic(const std::string& name);
|
||||
Topic get_topic(const std::string& name, TopicConfiguration config);
|
||||
@@ -26,6 +30,7 @@ protected:
|
||||
KafkaHandleBase(rd_kafka_t* handle);
|
||||
|
||||
void set_handle(rd_kafka_t* handle);
|
||||
void check_error(rd_kafka_resp_err_t error);
|
||||
private:
|
||||
using HandlePtr = std::unique_ptr<rd_kafka_t, decltype(&rd_kafka_destroy)>;
|
||||
|
||||
|
||||
@@ -117,10 +117,4 @@ void Consumer::commit(const TopicPartitionList& topic_partitions, bool async) {
|
||||
check_error(error);
|
||||
}
|
||||
|
||||
void Consumer::check_error(rd_kafka_resp_err_t error) {
|
||||
if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
|
||||
throw HandleException(error);
|
||||
}
|
||||
}
|
||||
|
||||
} // cppkafka
|
||||
|
||||
@@ -17,6 +17,18 @@ KafkaHandleBase::KafkaHandleBase(rd_kafka_t* handle)
|
||||
|
||||
}
|
||||
|
||||
void KafkaHandleBase::pause_partitions(const TopicPartitionList& topic_partitions) {
|
||||
rd_kafka_resp_err_t error = rd_kafka_pause_partitions(get_handle(),
|
||||
topic_partitions.get_handle());
|
||||
check_error(error);
|
||||
}
|
||||
|
||||
void KafkaHandleBase::resume_partitions(const TopicPartitionList& topic_partitions) {
|
||||
rd_kafka_resp_err_t error = rd_kafka_resume_partitions(get_handle(),
|
||||
topic_partitions.get_handle());
|
||||
check_error(error);
|
||||
}
|
||||
|
||||
rd_kafka_t* KafkaHandleBase::get_handle() {
|
||||
return handle_.get();
|
||||
}
|
||||
@@ -41,4 +53,10 @@ Topic KafkaHandleBase::get_topic(const string& name, rd_kafka_topic_conf_t* conf
|
||||
return Topic(topic);
|
||||
}
|
||||
|
||||
void KafkaHandleBase::check_error(rd_kafka_resp_err_t error) {
|
||||
if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
|
||||
throw HandleException(error);
|
||||
}
|
||||
}
|
||||
|
||||
} // cppkafka
|
||||
|
||||
Reference in New Issue
Block a user