diff --git a/include/cppkafka/consumer.h b/include/cppkafka/consumer.h index 8f49b6d..50bcb9c 100644 --- a/include/cppkafka/consumer.h +++ b/include/cppkafka/consumer.h @@ -347,8 +347,8 @@ public: * * The timeout used on this call will be the one configured via Consumer::set_timeout. * - * \return A message. The returned message *might* be empty. If's necessary to check - * that it's a valid one before using it: + * \return A message. The returned message *might* be empty. It's necessary to check + * that it's valid before using it: * * \code * Message msg = consumer.poll(); diff --git a/include/cppkafka/queue.h b/include/cppkafka/queue.h index bb31dcb..d7bc502 100644 --- a/include/cppkafka/queue.h +++ b/include/cppkafka/queue.h @@ -102,14 +102,14 @@ public: * * \param timeout The timeout to be set */ - void set_consume_timeout(std::chrono::milliseconds timeout); + void set_timeout(std::chrono::milliseconds timeout); /** * Gets the configured timeout. * * \sa Queue::set_timeout */ - std::chrono::milliseconds get_consume_timeout() const; + std::chrono::milliseconds get_timeout() const; /** * \brief Consume a message from this queue diff --git a/include/cppkafka/utils/roundrobin_poll_adapter.h b/include/cppkafka/utils/roundrobin_poll_adapter.h index 1988328..7a7fe9f 100644 --- a/include/cppkafka/utils/roundrobin_poll_adapter.h +++ b/include/cppkafka/utils/roundrobin_poll_adapter.h @@ -46,15 +46,8 @@ namespace cppkafka { * messages from each partition in turn. For performance reasons, librdkafka pre-fetches batches * of messages from the kafka broker (one batch from each partition), and stores them locally in * partition queues. Since all the internal partition queues are forwarded by default unto the - * group consumer queue (one per consumer), these batches end up being queued in sequence or arrival. - * For instance, a topic with 4 partitions (each containing N messages) will end up being queued as - * N1|N2|N3|N4 in the consumer queue. This means that for the Consumer to process messages from the - * 4th partition, it needs to consume 3xN messages. The larger the number of partitions, the more - * starvation occurs. While this behavior is acceptable for some applications, real-time applications - * sensitive to timing or those where messages must be processed more or less in the same order as - * they're being produced, the default librdkafka behavior is unacceptable. - * Fortunately, librdkafka exposes direct access to its partition queues which means that various - * polling strategies can be implemented to suit needs. + * group consumer queue (one per consumer), these batches end up being polled and consumed in the + * same sequence order. * This adapter allows fair round-robin polling of all assigned partitions, one message at a time * (or one batch at a time if poll_batch() is used). Note that poll_batch() has nothing to do with * the internal batching mechanism of librdkafka. @@ -64,6 +57,7 @@ namespace cppkafka { * \code * // Create a consumer * Consumer consumer(...); + * consumer.subscribe({ "my_topic" }); * * // Optionally set the callbacks. This must be done *BEFORE* creating the adapter * consumer.set_assignment_callback(...); @@ -73,9 +67,6 @@ namespace cppkafka { * // Create the adapter and use it for polling * RoundRobinPollAdapter adapter(consumer); * - * // Subscribe *AFTER* the adapter has been created - * consumer.subscribe({ "my_topic" }); - * * while (true) { * // Poll each partition in turn * Message msg = adapter.poll(); @@ -118,17 +109,16 @@ public: /** * \brief Polls all assigned partitions for new messages in round-robin fashion * - * Each call to poll() will result in another partition being polled. Aside from - * the partition, this function will also poll the main queue for events. If an - * event is found, it is immediately returned. As such the main queue has higher - * priority than the partition queues. Because of this, you - * need to call poll periodically as a keep alive mechanism, otherwise the broker - * will think this consumer is down and will trigger a rebalance (if using dynamic - * subscription). + * Each call to poll() will first consume from the global event queue and if there are + * no pending events, will attempt to consume from all partitions until a valid message is found. * The timeout used on this call will be the one configured via RoundRobinPollAdapter::set_timeout. * * \return A message. The returned message *might* be empty. It's necessary to check * that it's a valid one before using it (see example above). + * + * \remark You need to call poll() or poll_batch() periodically as a keep alive mechanism, + * otherwise the broker will think this consumer is down and will trigger a rebalance + * (if using dynamic subscription) */ Message poll(); @@ -145,55 +135,44 @@ public: /** * \brief Polls all assigned partitions for a batch of new messages in round-robin fashion * - * Each call to poll() will result in another partition being polled. Aside from - * the partition, this function will also poll the main queue for events. If a batch of - * events is found, it is prepended to the returned message list. If after polling the - * main queue the batch size has reached max_batch_size, it is immediately returned and - * the partition is no longer polled. Otherwise the partition is polled for the remaining - * messages up to the max_batch_size limit. - * Because of this, you need to call poll periodically as a keep alive mechanism, - * otherwise the broker will think this consumer is down and will trigger a rebalance - * (if using dynamic subscription). + * Each call to poll_batch() will first attempt to consume from the global event queue + * and if the maximum batch number has not yet been filled, will attempt to fill it by + * reading the remaining messages from each partition. * * \param max_batch_size The maximum amount of messages expected * * \return A list of messages + * + * \remark You need to call poll() or poll_batch() periodically as a keep alive mechanism, + * otherwise the broker will think this consumer is down and will trigger a rebalance + * (if using dynamic subscription) */ MessageList poll_batch(size_t max_batch_size); /** - * \brief Polls for a batch of messages depending on the configured PollStrategy + * \brief Polls all assigned partitions for a batch of new messages in round-robin fashion * * Same as the other overload of RoundRobinPollAdapter::poll_batch but the provided * timeout will be used instead of the one configured on this Consumer. * * \param max_batch_size The maximum amount of messages expected + * * \param timeout The timeout for this operation * * \return A list of messages */ MessageList poll_batch(size_t max_batch_size, std::chrono::milliseconds timeout); - /** - * \brief Gets the number of assigned partitions that can be polled across all topics - * - * \return The number of partitions - */ - size_t get_num_partitions(); - private: + void consume_batch(MessageList& messages, ssize_t& count, std::chrono::milliseconds timeout); + class CircularBuffer { public: - // typedefs - using toppar_t = std::pair; // - using qmap_t = std::map; - using qiter_t = qmap_t::iterator; - - qmap_t& ref() { + using QueueMap = std::map; + QueueMap& get_queues() { return queues_; } - - Queue& next() { + Queue& get_next_queue() { if (queues_.empty()) { throw QueueException(RD_KAFKA_RESP_ERR__STATE); } @@ -202,11 +181,10 @@ private: } return iter_->second; } - void rewind() { iter_ = queues_.begin(); } private: - qmap_t queues_; - qiter_t iter_ = queues_.begin(); + QueueMap queues_; + QueueMap::iterator iter_{queues_.begin()}; }; void on_assignment(TopicPartitionList& partitions); diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 71a14b7..77953d1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,4 +1,26 @@ -file(GLOB SOURCES *.cpp utils/*.cpp) +set(SOURCES + configuration.cpp + topic_configuration.cpp + configuration_option.cpp + exceptions.cpp + topic.cpp + buffer.cpp + queue.cpp + message.cpp + topic_partition.cpp + topic_partition_list.cpp + metadata.cpp + group_information.cpp + error.cpp + + kafka_handle_base.cpp + producer.cpp + consumer.cpp + + utils/backoff_performer.cpp + utils/backoff_committer.cpp + utils/roundrobin_poll_adapter.cpp +) include_directories(${CMAKE_CURRENT_SOURCE_DIR}/../include/cppkafka) include_directories(SYSTEM ${Boost_INCLUDE_DIRS} ${RDKAFKA_INCLUDE_DIR}) diff --git a/src/consumer.cpp b/src/consumer.cpp index eab195c..fe79b2a 100644 --- a/src/consumer.cpp +++ b/src/consumer.cpp @@ -259,19 +259,19 @@ vector Consumer::poll_batch(size_t max_batch_size, milliseconds timeout } Queue Consumer::get_main_queue() const { - Queue queue = Queue::make_non_owning(rd_kafka_queue_get_main(get_handle())); + Queue queue(rd_kafka_queue_get_main(get_handle())); queue.disable_queue_forwarding(); return queue; } Queue Consumer::get_consumer_queue() const { - return Queue::make_non_owning(rd_kafka_queue_get_consumer(get_handle())); + return rd_kafka_queue_get_consumer(get_handle()); } Queue Consumer::get_partition_queue(const TopicPartition& partition) const { - Queue queue = Queue::make_non_owning(rd_kafka_queue_get_partition(get_handle(), - partition.get_topic().c_str(), - partition.get_partition())); + Queue queue(rd_kafka_queue_get_partition(get_handle(), + partition.get_topic().c_str(), + partition.get_partition())); queue.disable_queue_forwarding(); return queue; } diff --git a/src/queue.cpp b/src/queue.cpp index af56715..410a84b 100644 --- a/src/queue.cpp +++ b/src/queue.cpp @@ -78,11 +78,11 @@ void Queue::disable_queue_forwarding() const { return rd_kafka_queue_forward(handle_.get(), nullptr); } -void Queue::set_consume_timeout(milliseconds timeout) { +void Queue::set_timeout(milliseconds timeout) { timeout_ms_ = timeout; } -milliseconds Queue::get_consume_timeout() const { +milliseconds Queue::get_timeout() const { return timeout_ms_; } @@ -99,7 +99,6 @@ MessageList Queue::consume_batch(size_t max_batch_size) const { } MessageList Queue::consume_batch(size_t max_batch_size, milliseconds timeout) const { - MessageList message_list; vector raw_message_list(max_batch_size); ssize_t num_messages = rd_kafka_consume_batch_queue(handle_.get(), static_cast(timeout.count()), @@ -110,14 +109,13 @@ MessageList Queue::consume_batch(size_t max_batch_size, milliseconds timeout) co if (error != RD_KAFKA_RESP_ERR_NO_ERROR) { throw QueueException(error); } - return message_list; + return MessageList(); } - raw_message_list.resize(num_messages); - message_list.reserve(num_messages); - for (auto&& message : raw_message_list) { - message_list.emplace_back(message); - } - return message_list; + // Build message list + MessageList messages; + messages.reserve(raw_message_list.size()); + messages.assign(raw_message_list.begin(), raw_message_list.end()); + return messages; } } //cppkafka diff --git a/src/utils/roundrobin_poll_adapter.cpp b/src/utils/roundrobin_poll_adapter.cpp index 64bb3fa..2df7e7d 100644 --- a/src/utils/roundrobin_poll_adapter.cpp +++ b/src/utils/roundrobin_poll_adapter.cpp @@ -30,7 +30,6 @@ #include "utils/roundrobin_poll_adapter.h" using std::string; -using std::make_pair; using std::chrono::milliseconds; using std::make_move_iterator; @@ -38,31 +37,31 @@ namespace cppkafka { RoundRobinPollAdapter::RoundRobinPollAdapter(Consumer& consumer) : consumer_(consumer), - assignment_callback_(consumer.get_assignment_callback()), - revocation_callback_(consumer.get_revocation_callback()), - rebalance_error_callback_(consumer.get_rebalance_error_callback()), consumer_queue_(consumer.get_consumer_queue()) { + // get all currently active partition assignments + TopicPartitionList assignment = consumer_.get_assignment(); + on_assignment(assignment); + // take over the assignment callback + assignment_callback_ = consumer.get_assignment_callback(); consumer_.set_assignment_callback([this](TopicPartitionList& partitions) { on_assignment(partitions); }); // take over the revocation callback + revocation_callback_ = consumer.get_revocation_callback(); consumer_.set_revocation_callback([this](const TopicPartitionList& partitions) { on_revocation(partitions); }); // take over the rebalance error callback + rebalance_error_callback_ = consumer.get_rebalance_error_callback(); consumer_.set_rebalance_error_callback([this](Error error) { on_rebalance_error(error); }); - // make sure we don't have any active subscriptions - if (!consumer_.get_subscription().empty()) { - throw ConsumerException(RD_KAFKA_RESP_ERR__EXISTING_SUBSCRIPTION); - } } RoundRobinPollAdapter::~RoundRobinPollAdapter() { restore_forwarding(); - //set the original callbacks + //reset the original callbacks consumer_.set_assignment_callback(assignment_callback_); consumer_.set_revocation_callback(revocation_callback_); consumer_.set_rebalance_error_callback(rebalance_error_callback_); @@ -81,17 +80,20 @@ Message RoundRobinPollAdapter::poll() { } Message RoundRobinPollAdapter::poll(milliseconds timeout) { - bool empty_list = partition_queues_.ref().empty(); - // Poll group event queue first - Message message = consumer_queue_.consume(empty_list ? timeout : milliseconds(0)); - if (message) { - return message; + size_t num_queues = partition_queues_.get_queues().size(); + // Always give priority to group and global events + Message message = consumer_queue_.consume(num_queues ? milliseconds(0) : timeout); + if (!message) { + while (num_queues--) { + //consume the next partition + message = partition_queues_.get_next_queue().consume(); + if (message) { + return message; + } + } } - if (!empty_list) { - //consume the next partition - message = partition_queues_.next().consume(timeout); - } - return message; + // wait on the next queue + return partition_queues_.get_next_queue().consume(timeout); } MessageList RoundRobinPollAdapter::poll_batch(size_t max_batch_size) { @@ -99,42 +101,42 @@ MessageList RoundRobinPollAdapter::poll_batch(size_t max_batch_size) { } MessageList RoundRobinPollAdapter::poll_batch(size_t max_batch_size, milliseconds timeout) { - bool empty_list = partition_queues_.ref().empty(); - ssize_t remaining_count = max_batch_size; + size_t num_queues = partition_queues_.get_queues().size(); + ssize_t count = max_batch_size; // batch from the group event queue first - MessageList messages = consumer_queue_.consume_batch(remaining_count, - empty_list ? timeout : milliseconds(0)); - remaining_count -= messages.size(); - if ((remaining_count <= 0) || empty_list) { - // the entire batch was filled - return messages; + MessageList messages = consumer_queue_.consume_batch(count, num_queues ? milliseconds(0) : timeout); + count -= messages.size(); + while ((count > 0) && (num_queues--)) { + // batch from the next partition + consume_batch(messages, count, milliseconds(0)); } - // batch from the next partition - MessageList partition_messages = partition_queues_.next().consume_batch(remaining_count, timeout); - if (messages.empty()) { - return partition_messages; + if (count > 0) { + // wait on the next queue + consume_batch(messages, count, timeout); } + return messages; +} + +void RoundRobinPollAdapter::consume_batch(MessageList& messages, ssize_t& count, milliseconds timeout) +{ + MessageList partition_messages = partition_queues_.get_next_queue().consume_batch(count, timeout); if (partition_messages.empty()) { - return messages; + return; } // concatenate both lists messages.reserve(messages.size() + partition_messages.size()); messages.insert(messages.end(), make_move_iterator(partition_messages.begin()), make_move_iterator(partition_messages.end())); - return messages; -} - -size_t RoundRobinPollAdapter::get_num_partitions() { - return partition_queues_.ref().size(); + // reduce total batch count + count -= partition_messages.size(); } void RoundRobinPollAdapter::on_assignment(TopicPartitionList& partitions) { // populate partition queues for (const auto& partition : partitions) { // get the queue associated with this partition - CircularBuffer::toppar_t key = make_pair(partition.get_topic(), partition.get_partition()); - partition_queues_.ref().emplace(key, consumer_.get_partition_queue(partition)); + partition_queues_.get_queues().emplace(partition, consumer_.get_partition_queue(partition)); } // reset the queue iterator partition_queues_.rewind(); @@ -147,13 +149,12 @@ void RoundRobinPollAdapter::on_assignment(TopicPartitionList& partitions) { void RoundRobinPollAdapter::on_revocation(const TopicPartitionList& partitions) { for (const auto& partition : partitions) { // get the queue associated with this partition - CircularBuffer::toppar_t key = make_pair(partition.get_topic(), partition.get_partition()); - auto qit = partition_queues_.ref().find(key); - if (qit != partition_queues_.ref().end()) { + auto qit = partition_queues_.get_queues().find(partition); + if (qit != partition_queues_.get_queues().end()) { // restore forwarding on this queue qit->second.forward_to_queue(consumer_queue_); // remove this queue from the list - partition_queues_.ref().erase(qit); + partition_queues_.get_queues().erase(qit); } } // reset the queue iterator @@ -174,7 +175,7 @@ void RoundRobinPollAdapter::on_rebalance_error(Error error) { void RoundRobinPollAdapter::restore_forwarding() { // forward all partition queues - for (const auto& toppar_queue : partition_queues_.ref()) { + for (const auto& toppar_queue : partition_queues_.get_queues()) { toppar_queue.second.forward_to_queue(consumer_queue_); } }