mirror of
https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
synced 2025-11-02 11:37:50 +00:00
Replace TopicPartitionList class with a vector of TopicPartition
This commit is contained in:
@@ -54,7 +54,7 @@ private:
|
||||
|
||||
void commit(const Message& msg, bool async);
|
||||
void commit(const TopicPartitionList& topic_partitions, bool async);
|
||||
void handle_rebalance(rd_kafka_resp_err_t err, const TopicPartitionList& topic_partitions);
|
||||
void handle_rebalance(rd_kafka_resp_err_t err, TopicPartitionList& topic_partitions);
|
||||
|
||||
AssignmentCallback assignment_callback_;
|
||||
RevocationCallback revocation_callback_;
|
||||
|
||||
@@ -5,8 +5,9 @@
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
#include <librdkafka/rdkafka.h>
|
||||
#include "topic_partition_list.h"
|
||||
#include "metadata.h"
|
||||
#include "topic_partition.h"
|
||||
#include "topic_partition_list.h"
|
||||
|
||||
namespace cppkafka {
|
||||
|
||||
|
||||
@@ -8,9 +8,11 @@ namespace cppkafka {
|
||||
|
||||
class TopicPartition {
|
||||
public:
|
||||
TopicPartition(const std::string& topic);
|
||||
TopicPartition(const std::string& topic, int partition);
|
||||
TopicPartition(const std::string& topic, int partition, int64_t offset);
|
||||
TopicPartition();
|
||||
TopicPartition(const char* topic);
|
||||
TopicPartition(std::string topic);
|
||||
TopicPartition(std::string topic, int partition);
|
||||
TopicPartition(std::string topic, int partition, int64_t offset);
|
||||
|
||||
const std::string& get_topic() const;
|
||||
int get_partition() const;
|
||||
|
||||
@@ -3,60 +3,19 @@
|
||||
|
||||
#include <memory>
|
||||
#include <algorithm>
|
||||
#include <initializer_list>
|
||||
#include <librdkafka/rdkafka.h>
|
||||
#include "topic_partition.h"
|
||||
|
||||
namespace cppkafka {
|
||||
|
||||
class TopicPartition;
|
||||
|
||||
class TopicPartitionList {
|
||||
public:
|
||||
static TopicPartitionList make_non_owning(rd_kafka_topic_partition_list_t* handle);
|
||||
using TopicPartitionsListPtr = std::unique_ptr<rd_kafka_topic_partition_list_t,
|
||||
decltype(&rd_kafka_topic_partition_list_destroy)>;
|
||||
using TopicPartitionList = std::vector<TopicPartition>;
|
||||
|
||||
TopicPartitionList();
|
||||
TopicPartitionList(const std::initializer_list<TopicPartition>& topic_partitions);
|
||||
TopicPartitionList(rd_kafka_topic_partition_list_t* handle);
|
||||
TopicPartitionList(size_t size);
|
||||
template <typename ForwardIterator>
|
||||
TopicPartitionList(ForwardIterator start, const ForwardIterator& end)
|
||||
: TopicPartitionList(std::distance(start, end)) {
|
||||
while (start != end) {
|
||||
add(*start);
|
||||
++start;
|
||||
}
|
||||
}
|
||||
TopicPartitionList(const TopicPartitionList& rhs);
|
||||
TopicPartitionList(TopicPartitionList&&) = default;
|
||||
TopicPartitionList& operator=(const TopicPartitionList& rhs);
|
||||
TopicPartitionList& operator=(TopicPartitionList&&) = default;
|
||||
|
||||
void add(const TopicPartition& topic_partition);
|
||||
void update(const TopicPartition& topic_partition);
|
||||
bool remove(const TopicPartition& topic_partition);
|
||||
|
||||
bool contains(const TopicPartition& topic_partition) const;
|
||||
size_t size() const;
|
||||
bool empty() const;
|
||||
|
||||
rd_kafka_topic_partition_list_t* get_handle() const;
|
||||
private:
|
||||
static const size_t DEFAULT_CONTAINER_SIZE;
|
||||
|
||||
struct NonOwningTag { };
|
||||
|
||||
using HandlePtr = std::unique_ptr<rd_kafka_topic_partition_list_t,
|
||||
decltype(&rd_kafka_topic_partition_list_destroy)>;
|
||||
|
||||
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;
|
||||
|
||||
HandlePtr handle_;
|
||||
};
|
||||
TopicPartitionsListPtr convert(const std::vector<TopicPartition>& topic_partitions);
|
||||
std::vector<TopicPartition> convert(const TopicPartitionsListPtr& topic_partitions);
|
||||
TopicPartitionsListPtr make_handle(rd_kafka_topic_partition_list_t* handle);
|
||||
|
||||
} // cppkafka
|
||||
|
||||
|
||||
Reference in New Issue
Block a user