mirror of
https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
synced 2025-11-03 12:07:57 +00:00
Added legacy offset store API
This commit is contained in:
@@ -291,6 +291,38 @@ public:
|
|||||||
* \return The topic partition list
|
* \return The topic partition list
|
||||||
*/
|
*/
|
||||||
TopicPartitionList get_offsets_position(const TopicPartitionList& topic_partitions) const;
|
TopicPartitionList get_offsets_position(const TopicPartitionList& topic_partitions) const;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* \brief Stores the offsets on the currently assigned topic/partitions (legacy).
|
||||||
|
*
|
||||||
|
* This translates into a call to rd_kafka_offsets_store with the current partition assignment.
|
||||||
|
* It is equivalent to calling rd_kafka_offsets_store(get_assignment()).
|
||||||
|
*
|
||||||
|
* \note When using this API it's recommended to set enable.auto.offset.store to false.
|
||||||
|
*/
|
||||||
|
void store_offsets() const;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* \brief Stores the offsets on the given topic/partitions (legacy).
|
||||||
|
*
|
||||||
|
* This translates into a call to rd_kafka_offsets_store.
|
||||||
|
*
|
||||||
|
* \param topic_partitions The topic/partition list to be stored.
|
||||||
|
*
|
||||||
|
* \note When using this API it's recommended to set enable.auto.offset.store to false.
|
||||||
|
*/
|
||||||
|
void store_offsets(const TopicPartitionList& topic_partitions) const;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* \brief Stores the offset for this message (legacy).
|
||||||
|
*
|
||||||
|
* This translates into a call to rd_kafka_offset_store.
|
||||||
|
*
|
||||||
|
* \param msg The message whose offset will be stored.
|
||||||
|
*
|
||||||
|
* \note When using this API it's recommended to set enable.auto.offset.store to false.
|
||||||
|
*/
|
||||||
|
void store_offset(const Message& msg) const;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* \brief Gets the current topic subscription
|
* \brief Gets the current topic subscription
|
||||||
|
|||||||
@@ -200,6 +200,28 @@ Consumer::get_offsets_position(const TopicPartitionList& topic_partitions) const
|
|||||||
return convert(topic_list_handle);
|
return convert(topic_list_handle);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Consumer::store_offsets() const
|
||||||
|
{
|
||||||
|
rd_kafka_topic_partition_list_t* list = nullptr;
|
||||||
|
rd_kafka_resp_err_t error = rd_kafka_assignment(get_handle(), &list);
|
||||||
|
check_error(error);
|
||||||
|
error = rd_kafka_offsets_store(get_handle(), list);
|
||||||
|
check_error(error, list);
|
||||||
|
}
|
||||||
|
|
||||||
|
void Consumer::store_offsets(const TopicPartitionList& topic_partitions) const
|
||||||
|
{
|
||||||
|
TopicPartitionsListPtr topic_list_handle = convert(topic_partitions);
|
||||||
|
rd_kafka_resp_err_t error = rd_kafka_offsets_store(get_handle(), topic_list_handle.get());
|
||||||
|
check_error(error, topic_list_handle.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
void Consumer::store_offset(const Message& msg) const
|
||||||
|
{
|
||||||
|
rd_kafka_resp_err_t error = rd_kafka_offset_store(msg.get_handle()->rkt, msg.get_partition(), msg.get_offset());
|
||||||
|
check_error(error);
|
||||||
|
}
|
||||||
|
|
||||||
vector<string> Consumer::get_subscription() const {
|
vector<string> Consumer::get_subscription() const {
|
||||||
rd_kafka_resp_err_t error;
|
rd_kafka_resp_err_t error;
|
||||||
rd_kafka_topic_partition_list_t* list = nullptr;
|
rd_kafka_topic_partition_list_t* list = nullptr;
|
||||||
|
|||||||
Reference in New Issue
Block a user