Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion include/cppkafka/consumer.h
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@ class CPPKAFKA_API Consumer : public KafkaHandleBase {
using AssignmentCallback = std::function<void(TopicPartitionList&)>;
using RevocationCallback = std::function<void(const TopicPartitionList&)>;
using RebalanceErrorCallback = std::function<void(Error)>;
using KafkaHandleBase::pause;

/**
* \brief Creates an instance of a consumer.
Expand Down Expand Up @@ -202,7 +203,7 @@ class CPPKAFKA_API Consumer : public KafkaHandleBase {
/**
* \brief Resumes all consumption
*/
void resume();
void resume();

/**
* \brief Commits the current partition assignment
Expand Down
14 changes: 14 additions & 0 deletions include/cppkafka/kafka_handle_base.h
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,13 @@ class CPPKAFKA_API KafkaHandleBase {
*/
void pause_partitions(const TopicPartitionList& topic_partitions);

/**
* \brief Pauses consumption/production for this topic
*
* \param topic The topic name
*/
void pause(const std::string& topic);

/**
* \brief Resumes consumption/production from the given topic/partition list
*
Expand All @@ -84,6 +91,13 @@ class CPPKAFKA_API KafkaHandleBase {
* \param topic_partitions The topic/partition list to resume consuming/producing from/to
*/
void resume_partitions(const TopicPartitionList& topic_partitions);

/**
* \brief Resumes consumption/production for this topic
*
* \param topic The topic name
*/
void resume(const std::string& topic);

/**
* \brief Sets the timeout for operations that require a timeout
Expand Down
1 change: 1 addition & 0 deletions include/cppkafka/producer.h
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ class Message;
*/
class CPPKAFKA_API Producer : public KafkaHandleBase {
public:
using KafkaHandleBase::pause;
/**
* The policy to use for the payload. The default policy is COPY_PAYLOAD
*/
Expand Down
3 changes: 3 additions & 0 deletions include/cppkafka/topic_partition_list.h
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
namespace cppkafka {

class TopicPartition;
class PartitionMetadata;

using TopicPartitionsListPtr = std::unique_ptr<rd_kafka_topic_partition_list_t,
decltype(&rd_kafka_topic_partition_list_destroy)>;
Expand All @@ -53,6 +54,8 @@ using TopicPartitionList = std::vector<TopicPartition>;
CPPKAFKA_API TopicPartitionsListPtr convert(const TopicPartitionList& topic_partitions);
CPPKAFKA_API TopicPartitionList convert(const TopicPartitionsListPtr& topic_partitions);
CPPKAFKA_API TopicPartitionList convert(rd_kafka_topic_partition_list_t* topic_partitions);
CPPKAFKA_API TopicPartitionList convert(const std::string& topic,
const std::vector<PartitionMetadata>& partition_metadata);
CPPKAFKA_API TopicPartitionsListPtr make_handle(rd_kafka_topic_partition_list_t* handle);

// Extracts a partition list subset belonging to the provided topics (case-insensitive)
Expand Down
8 changes: 8 additions & 0 deletions src/kafka_handle_base.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -64,13 +64,21 @@ void KafkaHandleBase::pause_partitions(const TopicPartitionList& topic_partition
check_error(error);
}

void KafkaHandleBase::pause(const std::string& topic) {
pause_partitions(convert(topic, get_metadata(get_topic(topic)).get_partitions()));
}

void KafkaHandleBase::resume_partitions(const TopicPartitionList& topic_partitions) {
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
rd_kafka_resp_err_t error = rd_kafka_resume_partitions(get_handle(),
topic_list_handle.get());
check_error(error);
}

void KafkaHandleBase::resume(const std::string& topic) {
resume_partitions(convert(topic, get_metadata(get_topic(topic)).get_partitions()));
}

void KafkaHandleBase::set_timeout(milliseconds timeout) {
timeout_ms_ = timeout;
}
Expand Down
11 changes: 11 additions & 0 deletions src/topic_partition_list.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
#include "topic_partition_list.h"
#include "topic_partition.h"
#include "exceptions.h"
#include "metadata.h"

using std::vector;
using std::set;
Expand Down Expand Up @@ -66,6 +67,16 @@ TopicPartitionList convert(rd_kafka_topic_partition_list_t* topic_partitions) {
return output;
}

TopicPartitionList convert(const std::string& topic,
const std::vector<PartitionMetadata>& partition_metadata)
{
TopicPartitionList output;
for (const auto& meta : partition_metadata) {
output.emplace_back(topic, meta.get_id());
}
return output;
}

TopicPartitionsListPtr make_handle(rd_kafka_topic_partition_list_t* handle) {
return TopicPartitionsListPtr(handle, &rd_kafka_topic_partition_list_destroy);
}
Expand Down