Add metadata objects and tests for them

This commit is contained in:
Matias Fontanini
2016-05-26 20:45:12 -07:00
parent 6374062f47
commit c31d00115a
10 changed files with 344 additions and 17 deletions

View File

@@ -8,6 +8,7 @@ set(SOURCES
message.cpp
topic_partition.cpp
topic_partition_list.cpp
metadata.cpp
kafka_handle_base.cpp
producer.cpp

View File

@@ -10,16 +10,13 @@ using std::chrono::milliseconds;
namespace cppkafka {
const milliseconds Consumer::DEFAULT_TIMEOUT{1000};
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) {
Consumer::Consumer(Configuration config) {
char error_buffer[512];
// Set ourselves as the opaque pointer
rd_kafka_conf_set_opaque(config.get_handle(), this);
@@ -37,10 +34,6 @@ Consumer::~Consumer() {
close();
}
void Consumer::set_timeout(const milliseconds timeout) {
timeout_ms_ = timeout;
}
void Consumer::set_assignment_callback(AssignmentCallback callback) {
assignment_callback_ = move(callback);
}
@@ -101,7 +94,7 @@ TopicPartitionList Consumer::get_offsets_committed(const TopicPartitionList& top
// Copy the list, let rd_kafka change it and return it
TopicPartitionList output = topic_partitions;
rd_kafka_resp_err_t error = rd_kafka_committed(get_handle(), output.get_handle(),
timeout_ms_.count());
get_timeout().count());
check_error(error);
return output;
}
@@ -131,7 +124,8 @@ TopicPartitionList Consumer::get_assignment() {
}
Message Consumer::poll() {
rd_kafka_message_t* message = rd_kafka_consumer_poll(get_handle(), timeout_ms_.count());
rd_kafka_message_t* message = rd_kafka_consumer_poll(get_handle(),
get_timeout().count());
return message ? Message(message) : Message();
}

View File

@@ -4,16 +4,19 @@
#include "topic.h"
using std::string;
using std::chrono::milliseconds;
namespace cppkafka {
const milliseconds KafkaHandleBase::DEFAULT_TIMEOUT{1000};
KafkaHandleBase::KafkaHandleBase()
: handle_(nullptr, nullptr) {
: handle_(nullptr, nullptr), timeout_ms_(DEFAULT_TIMEOUT) {
}
KafkaHandleBase::KafkaHandleBase(rd_kafka_t* handle)
: handle_(handle, &rd_kafka_destroy) {
: handle_(handle, &rd_kafka_destroy), timeout_ms_(DEFAULT_TIMEOUT) {
}
@@ -29,6 +32,10 @@ void KafkaHandleBase::resume_partitions(const TopicPartitionList& topic_partitio
check_error(error);
}
void KafkaHandleBase::set_timeout(const milliseconds& timeout) {
timeout_ms_ = timeout;
}
rd_kafka_t* KafkaHandleBase::get_handle() {
return handle_.get();
}
@@ -41,6 +48,18 @@ Topic KafkaHandleBase::get_topic(const string& name, TopicConfiguration topicCon
return get_topic(name, topicConfig.get_handle());
}
Metadata KafkaHandleBase::get_metadata() {
return get_metadata(nullptr);
}
Metadata KafkaHandleBase::get_metadata(const Topic& topic) {
return get_metadata(topic.get_handle());
}
milliseconds KafkaHandleBase::get_timeout() const {
return timeout_ms_;
}
void KafkaHandleBase::set_handle(rd_kafka_t* handle) {
handle_ = HandlePtr(handle, &rd_kafka_destroy);
}
@@ -53,6 +72,14 @@ Topic KafkaHandleBase::get_topic(const string& name, rd_kafka_topic_conf_t* conf
return Topic(topic);
}
Metadata KafkaHandleBase::get_metadata(rd_kafka_topic_t* topic_ptr) {
const rd_kafka_metadata_t* metadata;
rd_kafka_resp_err_t error = rd_kafka_metadata(get_handle(), topic_ptr != nullptr,
topic_ptr, &metadata, timeout_ms_.count());
check_error(error);
return Metadata(metadata);
}
void KafkaHandleBase::check_error(rd_kafka_resp_err_t error) {
if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
throw HandleException(error);

129
src/metadata.cpp Normal file
View File

@@ -0,0 +1,129 @@
#include "metadata.h"
using std::string;
using std::vector;
using std::unordered_set;
namespace cppkafka {
// PartitionMetadata
PartitionMetadata::PartitionMetadata(const rd_kafka_metadata_partition& partition)
: id_(partition.id), error_(partition.err), leader_(partition.leader) {
for (int i = 0; i < partition.replica_cnt; ++i) {
replicas_.push_back(partition.replicas[i]);
}
for (int i = 0; i < partition.isr_cnt; ++i) {
isrs_.push_back(partition.isrs[i]);
}
}
uint32_t PartitionMetadata::get_id() const {
return id_;
}
rd_kafka_resp_err_t PartitionMetadata::get_error() const {
return error_;
}
int32_t PartitionMetadata::get_leader() const {
return leader_;
}
const vector<int32_t>& PartitionMetadata::get_replicas() const {
return replicas_;
}
const vector<int32_t>& PartitionMetadata::get_in_sync_replica_brokers() const {
return isrs_;
}
// TopicMetadata
TopicMetadata::TopicMetadata(const rd_kafka_metadata_topic& topic)
: topic_(topic.topic), error_(topic.err) {
for (int i = 0; i < topic.partition_cnt; ++i) {
partitions_.emplace_back(topic.partitions[i]);
}
}
const string& TopicMetadata::get_topic() const {
return topic_;
}
rd_kafka_resp_err_t TopicMetadata::get_error() const {
return error_;
}
const vector<PartitionMetadata>& TopicMetadata::get_partitions() const {
return partitions_;
}
// BrokerMetadata
BrokerMetadata::BrokerMetadata(const rd_kafka_metadata_broker_t& broker)
: host_(broker.host), id_(broker.id), port_(static_cast<uint16_t>(broker.port)) {
}
const string& BrokerMetadata::get_host() const {
return host_;
}
int32_t BrokerMetadata::get_id() const {
return id_;
}
uint16_t BrokerMetadata::get_port() const {
return port_;
}
// Metadata
Metadata::Metadata(const rd_kafka_metadata_t* ptr)
: handle_(ptr, &rd_kafka_metadata_destroy) {
}
vector<BrokerMetadata> Metadata::get_brokers() const {
vector<BrokerMetadata> output;
for (int i = 0; i < handle_->broker_cnt; ++i) {
const rd_kafka_metadata_broker_t& broker = handle_->brokers[i];
output.emplace_back(broker);
}
return output;
}
vector<TopicMetadata> Metadata::get_topics() const {
vector<TopicMetadata> output;
for (int i = 0; i < handle_->topic_cnt; ++i) {
const rd_kafka_metadata_topic_t& topic = handle_->topics[i];
output.emplace_back(topic);
}
return output;
}
vector<TopicMetadata> Metadata::get_topics(const unordered_set<string>& topics) const {
vector<TopicMetadata> output;
for (int i = 0; i < handle_->topic_cnt; ++i) {
const rd_kafka_metadata_topic_t& topic = handle_->topics[i];
if (topics.count(topic.topic)) {
output.emplace_back(topic);
}
}
return output;
}
vector<TopicMetadata> Metadata::get_topics(const string& prefix) const {
vector<TopicMetadata> output;
for (int i = 0; i < handle_->topic_cnt; ++i) {
const rd_kafka_metadata_topic_t& topic = handle_->topics[i];
string topic_name = topic.topic;
if (topic_name.find(prefix) == 0) {
output.emplace_back(topic);
}
}
return output;
}
} // cppkafka