Replace TopicPartitionList class with a vector of TopicPartition

This commit is contained in:
Matias Fontanini
2016-05-28 09:02:44 -07:00
parent c31d00115a
commit 5b957de7e4
10 changed files with 106 additions and 171 deletions

View File

@@ -10,9 +10,15 @@ using std::chrono::milliseconds;
namespace cppkafka {
void dummy_topic_partition_list_deleter(rd_kafka_topic_partition_list_t*) {
}
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);
// Build a dummy unique_ptr which won't actually delete the ptr
TopicPartitionsListPtr handle(partitions, &dummy_topic_partition_list_deleter);
TopicPartitionList list = convert(handle);
static_cast<Consumer*>(opaque)->handle_rebalance(error, list);
}
@@ -47,8 +53,9 @@ void Consumer::set_rebalance_error_callback(RebalanceErrorCallback callback) {
}
void Consumer::subscribe(const vector<string>& topics) {
TopicPartitionList list(topics.begin(), topics.end());
rd_kafka_resp_err_t error = rd_kafka_subscribe(get_handle(), list.get_handle());
TopicPartitionList topic_partitions(topics.begin(), topics.end());
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_subscribe(get_handle(), topic_list_handle.get());
check_error(error);
}
@@ -58,8 +65,9 @@ void Consumer::unsubscribe() {
}
void Consumer::assign(const TopicPartitionList& topic_partitions) {
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
// If the list is empty, then we need to use a null pointer
auto handle = topic_partitions.empty() ? nullptr : topic_partitions.get_handle();
auto handle = topic_partitions.empty() ? nullptr : topic_list_handle.get();
rd_kafka_resp_err_t error = rd_kafka_assign(get_handle(), handle);
check_error(error);
}
@@ -91,20 +99,18 @@ void Consumer::async_commit(const TopicPartitionList& topic_partitions) {
}
TopicPartitionList Consumer::get_offsets_committed(const TopicPartitionList& topic_partitions) {
// 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(),
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_committed(get_handle(), topic_list_handle.get(),
get_timeout().count());
check_error(error);
return output;
return convert(topic_list_handle);
}
TopicPartitionList Consumer::get_offsets_position(const TopicPartitionList& topic_partitions) {
// Copy the list, let rd_kafka change it and return it
TopicPartitionList output = topic_partitions;
rd_kafka_resp_err_t error = rd_kafka_position(get_handle(), output.get_handle());
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_position(get_handle(), topic_list_handle.get());
check_error(error);
return output;
return convert(topic_list_handle);
}
TopicPartitionList Consumer::get_subscription() {
@@ -112,7 +118,7 @@ TopicPartitionList Consumer::get_subscription() {
rd_kafka_topic_partition_list_t* list = nullptr;
error = rd_kafka_subscription(get_handle(), &list);
check_error(error);
return TopicPartitionList(list);
return convert(make_handle(list));
}
TopicPartitionList Consumer::get_assignment() {
@@ -120,7 +126,7 @@ TopicPartitionList Consumer::get_assignment() {
rd_kafka_topic_partition_list_t* list = nullptr;
error = rd_kafka_assignment(get_handle(), &list);
check_error(error);
return TopicPartitionList(list);
return convert(make_handle(list));
}
Message Consumer::poll() {
@@ -137,13 +143,14 @@ void Consumer::commit(const Message& msg, bool async) {
}
void Consumer::commit(const TopicPartitionList& topic_partitions, bool async) {
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error;
error = rd_kafka_commit(get_handle(), topic_partitions.get_handle(), async ? 1 : 0);
error = rd_kafka_commit(get_handle(), topic_list_handle.get(), async ? 1 : 0);
check_error(error);
}
void Consumer::handle_rebalance(rd_kafka_resp_err_t error,
const TopicPartitionList& topic_partitions) {
TopicPartitionList& topic_partitions) {
if (error == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) {
if (assignment_callback_) {
assignment_callback_(topic_partitions);