2019-01-21 14:02:03 +00:00
|
|
|
#pragma once
|
|
|
|
|
2019-04-22 13:23:05 +00:00
|
|
|
#include <Core/Names.h>
|
2019-05-22 19:38:43 +00:00
|
|
|
#include <Core/Types.h>
|
2019-09-04 21:25:33 +00:00
|
|
|
#include <IO/ReadBuffer.h>
|
2019-01-21 14:02:03 +00:00
|
|
|
|
|
|
|
#include <cppkafka/cppkafka.h>
|
|
|
|
|
2019-09-04 21:25:33 +00:00
|
|
|
namespace Poco
|
|
|
|
{
|
|
|
|
class Logger;
|
|
|
|
}
|
|
|
|
|
2019-01-21 14:02:03 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2019-04-22 13:23:05 +00:00
|
|
|
|
2019-01-21 14:02:03 +00:00
|
|
|
using ConsumerPtr = std::shared_ptr<cppkafka::Consumer>;
|
|
|
|
|
|
|
|
class ReadBufferFromKafkaConsumer : public ReadBuffer
|
|
|
|
{
|
|
|
|
public:
|
2019-05-16 15:20:30 +00:00
|
|
|
ReadBufferFromKafkaConsumer(
|
2019-08-29 15:36:07 +00:00
|
|
|
ConsumerPtr consumer_,
|
|
|
|
Poco::Logger * log_,
|
|
|
|
size_t max_batch_size,
|
|
|
|
size_t poll_timeout_,
|
|
|
|
bool intermediate_commit_,
|
2020-01-30 19:30:45 +00:00
|
|
|
const std::atomic<bool> & stopped_,
|
|
|
|
const Names & _topics
|
|
|
|
);
|
2019-05-27 17:25:34 +00:00
|
|
|
~ReadBufferFromKafkaConsumer() override;
|
2019-01-21 14:02:03 +00:00
|
|
|
|
2019-09-20 12:12:32 +00:00
|
|
|
void allowNext() { allowed = true; } // Allow to read next message.
|
2019-04-22 13:23:05 +00:00
|
|
|
void commit(); // Commit all processed messages.
|
2020-01-30 19:30:45 +00:00
|
|
|
void subscribe(); // Subscribe internal consumer to topics.
|
2019-04-22 13:23:05 +00:00
|
|
|
void unsubscribe(); // Unsubscribe internal consumer in case of failure.
|
2019-01-21 14:02:03 +00:00
|
|
|
|
2019-07-19 15:01:34 +00:00
|
|
|
auto pollTimeout() const { return poll_timeout; }
|
2019-05-14 15:52:03 +00:00
|
|
|
|
2020-01-28 14:21:36 +00:00
|
|
|
bool hasMorePolledMessages() const;
|
2020-01-30 19:30:45 +00:00
|
|
|
auto rebalanceHappened() const { return rebalance_happened; }
|
|
|
|
|
|
|
|
void storeLastReadMessageOffset();
|
|
|
|
void resetToLastCommitted(const char * msg);
|
2020-01-28 14:21:36 +00:00
|
|
|
|
2019-05-22 19:38:43 +00:00
|
|
|
// Return values for the message that's being read.
|
2019-05-23 11:15:18 +00:00
|
|
|
String currentTopic() const { return current[-1].get_topic(); }
|
|
|
|
String currentKey() const { return current[-1].get_key(); }
|
|
|
|
auto currentOffset() const { return current[-1].get_offset(); }
|
2019-07-22 11:32:11 +00:00
|
|
|
auto currentPartition() const { return current[-1].get_partition(); }
|
2019-08-06 14:18:37 +00:00
|
|
|
auto currentTimestamp() const { return current[-1].get_timestamp(); }
|
2019-05-22 19:38:43 +00:00
|
|
|
|
2019-01-21 14:02:03 +00:00
|
|
|
private:
|
2019-01-25 12:48:59 +00:00
|
|
|
using Messages = std::vector<cppkafka::Message>;
|
|
|
|
|
2019-01-21 14:02:03 +00:00
|
|
|
ConsumerPtr consumer;
|
|
|
|
Poco::Logger * log;
|
2019-01-25 12:48:59 +00:00
|
|
|
const size_t batch_size = 1;
|
2019-05-14 15:52:03 +00:00
|
|
|
const size_t poll_timeout = 0;
|
2020-01-30 19:30:45 +00:00
|
|
|
size_t offsets_stored = 0;
|
2019-05-15 16:11:50 +00:00
|
|
|
bool stalled = false;
|
2019-05-16 15:20:30 +00:00
|
|
|
bool intermediate_commit = true;
|
2019-09-20 12:12:32 +00:00
|
|
|
bool allowed = true;
|
2019-09-04 21:25:33 +00:00
|
|
|
|
2019-08-29 15:36:07 +00:00
|
|
|
const std::atomic<bool> & stopped;
|
|
|
|
|
2019-01-25 12:48:59 +00:00
|
|
|
Messages messages;
|
|
|
|
Messages::const_iterator current;
|
2019-01-21 14:02:03 +00:00
|
|
|
|
2020-01-30 19:30:45 +00:00
|
|
|
bool rebalance_happened = false;
|
|
|
|
cppkafka::TopicPartitionList assignment;
|
|
|
|
const Names topics;
|
|
|
|
|
2019-01-21 14:02:03 +00:00
|
|
|
bool nextImpl() override;
|
|
|
|
};
|
|
|
|
|
2019-04-22 13:23:05 +00:00
|
|
|
}
|