mirror of
				https://github.com/Telecominfraproject/wlan-cloud-lib-cppkafka.git
				synced 2025-11-04 12:37:54 +00:00 
			
		
		
		
	Allocators (#118)
* Added allocator support for consumers and buffered producer * Changed MessageList back to std::vector<Message> for consistency with the allocator API
This commit is contained in:
		
				
					committed by
					
						
						Matias Fontanini
					
				
			
			
				
	
			
			
			
						parent
						
							d77e7466b8
						
					
				
				
					commit
					9af4330c6d
				
			@@ -379,10 +379,14 @@ public:
 | 
				
			|||||||
     * This can return one or more messages
 | 
					     * This can return one or more messages
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
     * \param max_batch_size The maximum amount of messages expected
 | 
					     * \param max_batch_size The maximum amount of messages expected
 | 
				
			||||||
 | 
					     * \param alloc The optionally supplied allocator for allocating messages
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
     * \return A list of messages
 | 
					     * \return A list of messages
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size);
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                               const Allocator& alloc);
 | 
				
			||||||
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size);
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \brief Polls for a batch of messages
 | 
					     * \brief Polls for a batch of messages
 | 
				
			||||||
@@ -391,10 +395,16 @@ public:
 | 
				
			|||||||
     *
 | 
					     *
 | 
				
			||||||
     * \param max_batch_size The maximum amount of messages expected
 | 
					     * \param max_batch_size The maximum amount of messages expected
 | 
				
			||||||
     * \param timeout The timeout for this operation
 | 
					     * \param timeout The timeout for this operation
 | 
				
			||||||
 | 
					     * \param alloc The optionally supplied allocator for allocating messages
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
     * \return A list of messages
 | 
					     * \return A list of messages
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size, std::chrono::milliseconds timeout);
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                               std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                               const Allocator& alloc);
 | 
				
			||||||
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                    std::chrono::milliseconds timeout);
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \brief Get the global event queue servicing this consumer corresponding to
 | 
					     * \brief Get the global event queue servicing this consumer corresponding to
 | 
				
			||||||
@@ -430,6 +440,7 @@ public:
 | 
				
			|||||||
private:
 | 
					private:
 | 
				
			||||||
    static void rebalance_proxy(rd_kafka_t *handle, rd_kafka_resp_err_t error,
 | 
					    static void rebalance_proxy(rd_kafka_t *handle, rd_kafka_resp_err_t error,
 | 
				
			||||||
                                rd_kafka_topic_partition_list_t *partitions, void *opaque);
 | 
					                                rd_kafka_topic_partition_list_t *partitions, void *opaque);
 | 
				
			||||||
 | 
					    static Queue get_queue(rd_kafka_queue_t* handle);
 | 
				
			||||||
    void close();
 | 
					    void close();
 | 
				
			||||||
    void commit(const Message& msg, bool async);
 | 
					    void commit(const Message& msg, bool async);
 | 
				
			||||||
    void commit(const TopicPartitionList* topic_partitions, bool async);
 | 
					    void commit(const TopicPartitionList* topic_partitions, bool async);
 | 
				
			||||||
@@ -440,6 +451,30 @@ private:
 | 
				
			|||||||
    RebalanceErrorCallback rebalance_error_callback_;
 | 
					    RebalanceErrorCallback rebalance_error_callback_;
 | 
				
			||||||
};
 | 
					};
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// Implementations
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> Consumer::poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                     const Allocator& alloc) {
 | 
				
			||||||
 | 
					    return poll_batch(max_batch_size, get_timeout(), alloc);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> Consumer::poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                     std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                                     const Allocator& alloc) {
 | 
				
			||||||
 | 
					    std::vector<rd_kafka_message_t*> raw_messages(max_batch_size);
 | 
				
			||||||
 | 
					    // Note that this will leak the queue when using rdkafka < 0.11.5 (see get_queue comment)
 | 
				
			||||||
 | 
					    Queue queue(get_queue(rd_kafka_queue_get_consumer(get_handle())));
 | 
				
			||||||
 | 
					    ssize_t result = rd_kafka_consume_batch_queue(queue.get_handle() , timeout.count(), raw_messages.data(),
 | 
				
			||||||
 | 
					                                                  raw_messages.size());
 | 
				
			||||||
 | 
					    if (result == -1) {
 | 
				
			||||||
 | 
					        check_error(rd_kafka_last_error());
 | 
				
			||||||
 | 
					        // on the off-chance that check_error() does not throw an error
 | 
				
			||||||
 | 
					        return std::vector<Message, Allocator>(alloc);
 | 
				
			||||||
 | 
					    }
 | 
				
			||||||
 | 
					    return std::vector<Message, Allocator>(raw_messages.begin(), raw_messages.begin() + result, alloc);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
} // cppkafka
 | 
					} // cppkafka
 | 
				
			||||||
 | 
					
 | 
				
			||||||
#endif // CPP_KAFKA_CONSUMER_H
 | 
					#endif // CPP_KAFKA_CONSUMER_H
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -138,9 +138,14 @@ public:
 | 
				
			|||||||
     *
 | 
					     *
 | 
				
			||||||
     * \param max_batch_size The max number of messages to consume if available
 | 
					     * \param max_batch_size The max number of messages to consume if available
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
 | 
					     * \param alloc The optionally supplied allocator for the message list
 | 
				
			||||||
 | 
					     *
 | 
				
			||||||
     * \return A list of messages. Could be empty if there's nothing to consume
 | 
					     * \return A list of messages. Could be empty if there's nothing to consume
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList consume_batch(size_t max_batch_size) const;
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> consume_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                  const Allocator& alloc) const;
 | 
				
			||||||
 | 
					    std::vector<Message> consume_batch(size_t max_batch_size) const;
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \brief Consumes a batch of messages from this queue
 | 
					     * \brief Consumes a batch of messages from this queue
 | 
				
			||||||
@@ -151,9 +156,16 @@ public:
 | 
				
			|||||||
     *
 | 
					     *
 | 
				
			||||||
     * \param timeout The timeout to be used on this call
 | 
					     * \param timeout The timeout to be used on this call
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
 | 
					     * \param alloc The optionally supplied allocator for the message list
 | 
				
			||||||
 | 
					     *
 | 
				
			||||||
     * \return A list of messages. Could be empty if there's nothing to consume
 | 
					     * \return A list of messages. Could be empty if there's nothing to consume
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList consume_batch(size_t max_batch_size, std::chrono::milliseconds timeout) const;
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> consume_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                  std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                                  const Allocator& alloc) const;
 | 
				
			||||||
 | 
					    std::vector<Message> consume_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                       std::chrono::milliseconds timeout) const;
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * Indicates whether this queue is valid (not null)
 | 
					     * Indicates whether this queue is valid (not null)
 | 
				
			||||||
@@ -178,6 +190,32 @@ private:
 | 
				
			|||||||
 | 
					
 | 
				
			||||||
using QueueList = std::vector<Queue>;
 | 
					using QueueList = std::vector<Queue>;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> Queue::consume_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                     const Allocator& alloc) const {
 | 
				
			||||||
 | 
					    return consume_batch(max_batch_size, timeout_ms_, alloc);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> Queue::consume_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                     std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                                     const Allocator& alloc) const {
 | 
				
			||||||
 | 
					    std::vector<rd_kafka_message_t*> raw_messages(max_batch_size);
 | 
				
			||||||
 | 
					    ssize_t result = rd_kafka_consume_batch_queue(handle_.get(),
 | 
				
			||||||
 | 
					                                                  static_cast<int>(timeout.count()),
 | 
				
			||||||
 | 
					                                                  raw_messages.data(),
 | 
				
			||||||
 | 
					                                                  raw_messages.size());
 | 
				
			||||||
 | 
					    if (result == -1) {
 | 
				
			||||||
 | 
					        rd_kafka_resp_err_t error = rd_kafka_last_error();
 | 
				
			||||||
 | 
					        if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
 | 
				
			||||||
 | 
					            throw QueueException(error);
 | 
				
			||||||
 | 
					        }
 | 
				
			||||||
 | 
					        return std::vector<Message, Allocator>(alloc);
 | 
				
			||||||
 | 
					    }
 | 
				
			||||||
 | 
					    // Build message list
 | 
				
			||||||
 | 
					    return std::vector<Message, Allocator>(raw_messages.begin(), raw_messages.begin() + result, alloc);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
} // cppkafka
 | 
					} // cppkafka
 | 
				
			||||||
 | 
					
 | 
				
			||||||
#endif //CPPKAFKA_QUEUE_H
 | 
					#endif //CPPKAFKA_QUEUE_H
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -83,7 +83,8 @@ namespace cppkafka {
 | 
				
			|||||||
 * the messages *after* the ProduceSuccessCallback has reported a successful delivery to avoid memory
 | 
					 * the messages *after* the ProduceSuccessCallback has reported a successful delivery to avoid memory
 | 
				
			||||||
 * corruptions.
 | 
					 * corruptions.
 | 
				
			||||||
 */
 | 
					 */
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType,
 | 
				
			||||||
 | 
					          typename Allocator = std::allocator<ConcreteMessageBuilder<BufferType>>>
 | 
				
			||||||
class CPPKAFKA_API BufferedProducer {
 | 
					class CPPKAFKA_API BufferedProducer {
 | 
				
			||||||
public:
 | 
					public:
 | 
				
			||||||
    enum class FlushMethod { Sync,    ///< Empty the buffer and wait for acks from the broker
 | 
					    enum class FlushMethod { Sync,    ///< Empty the buffer and wait for acks from the broker
 | 
				
			||||||
@@ -92,6 +93,7 @@ public:
 | 
				
			|||||||
     * Concrete builder
 | 
					     * Concrete builder
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    using Builder = ConcreteMessageBuilder<BufferType>;
 | 
					    using Builder = ConcreteMessageBuilder<BufferType>;
 | 
				
			||||||
 | 
					    using QueueType = std::deque<Builder, Allocator>;
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * Callback to indicate a message was delivered to the broker
 | 
					     * Callback to indicate a message was delivered to the broker
 | 
				
			||||||
@@ -115,8 +117,9 @@ public:
 | 
				
			|||||||
     * \brief Constructs a buffered producer using the provided configuration
 | 
					     * \brief Constructs a buffered producer using the provided configuration
 | 
				
			||||||
     *
 | 
					     *
 | 
				
			||||||
     * \param config The configuration to be used on the actual Producer object
 | 
					     * \param config The configuration to be used on the actual Producer object
 | 
				
			||||||
 | 
					     * \param alloc The optionally supplied allocator for the internal message buffer
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    BufferedProducer(Configuration config);
 | 
					    BufferedProducer(Configuration config, const Allocator& alloc = Allocator());
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \brief Adds a message to the producer's buffer.
 | 
					     * \brief Adds a message to the producer's buffer.
 | 
				
			||||||
@@ -390,7 +393,6 @@ protected:
 | 
				
			|||||||
#endif
 | 
					#endif
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
private:
 | 
					private:
 | 
				
			||||||
    using QueueType = std::deque<Builder>;
 | 
					 | 
				
			||||||
    enum class MessagePriority { Low, High };
 | 
					    enum class MessagePriority { Low, High };
 | 
				
			||||||
    enum class SenderType { Sync, Async };
 | 
					    enum class SenderType { Sync, Async };
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
@@ -466,28 +468,30 @@ Producer::PayloadPolicy get_default_payload_policy<Buffer>() {
 | 
				
			|||||||
    return Producer::PayloadPolicy::PASSTHROUGH_PAYLOAD;
 | 
					    return Producer::PayloadPolicy::PASSTHROUGH_PAYLOAD;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
BufferedProducer<BufferType>::BufferedProducer(Configuration config)
 | 
					BufferedProducer<BufferType, Allocator>::BufferedProducer(Configuration config,
 | 
				
			||||||
: producer_(prepare_configuration(std::move(config))) {
 | 
					                                                          const Allocator& alloc)
 | 
				
			||||||
 | 
					: producer_(prepare_configuration(std::move(config))),
 | 
				
			||||||
 | 
					  messages_(alloc) {
 | 
				
			||||||
    producer_.set_payload_policy(get_default_payload_policy<BufferType>());
 | 
					    producer_.set_payload_policy(get_default_payload_policy<BufferType>());
 | 
				
			||||||
#ifdef KAFKA_TEST_INSTANCE
 | 
					#ifdef KAFKA_TEST_INSTANCE
 | 
				
			||||||
    test_params_ = nullptr;
 | 
					    test_params_ = nullptr;
 | 
				
			||||||
#endif
 | 
					#endif
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::add_message(const MessageBuilder& builder) {
 | 
					void BufferedProducer<BufferType, Allocator>::add_message(const MessageBuilder& builder) {
 | 
				
			||||||
    add_message(Builder(builder)); //make ConcreteBuilder
 | 
					    add_message(Builder(builder)); //make ConcreteBuilder
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::add_message(Builder builder) {
 | 
					void BufferedProducer<BufferType, Allocator>::add_message(Builder builder) {
 | 
				
			||||||
    add_tracker(SenderType::Async, builder);
 | 
					    add_tracker(SenderType::Async, builder);
 | 
				
			||||||
    do_add_message(move(builder), MessagePriority::Low, true);
 | 
					    do_add_message(move(builder), MessagePriority::Low, true);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::produce(const MessageBuilder& builder) {
 | 
					void BufferedProducer<BufferType, Allocator>::produce(const MessageBuilder& builder) {
 | 
				
			||||||
    if (has_internal_data_) {
 | 
					    if (has_internal_data_) {
 | 
				
			||||||
        MessageBuilder builder_clone(builder.clone());
 | 
					        MessageBuilder builder_clone(builder.clone());
 | 
				
			||||||
        add_tracker(SenderType::Async, builder_clone);
 | 
					        add_tracker(SenderType::Async, builder_clone);
 | 
				
			||||||
@@ -498,8 +502,8 @@ void BufferedProducer<BufferType>::produce(const MessageBuilder& builder) {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::sync_produce(const MessageBuilder& builder) {
 | 
					void BufferedProducer<BufferType, Allocator>::sync_produce(const MessageBuilder& builder) {
 | 
				
			||||||
    if (has_internal_data_) {
 | 
					    if (has_internal_data_) {
 | 
				
			||||||
        MessageBuilder builder_clone(builder.clone());
 | 
					        MessageBuilder builder_clone(builder.clone());
 | 
				
			||||||
        TrackerPtr tracker = add_tracker(SenderType::Sync, builder_clone);
 | 
					        TrackerPtr tracker = add_tracker(SenderType::Sync, builder_clone);
 | 
				
			||||||
@@ -519,13 +523,13 @@ void BufferedProducer<BufferType>::sync_produce(const MessageBuilder& builder) {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::produce(const Message& message) {
 | 
					void BufferedProducer<BufferType, Allocator>::produce(const Message& message) {
 | 
				
			||||||
    async_produce(MessageBuilder(message), true);
 | 
					    async_produce(MessageBuilder(message), true);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::async_flush() {
 | 
					void BufferedProducer<BufferType, Allocator>::async_flush() {
 | 
				
			||||||
    CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
					    CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
				
			||||||
    QueueType flush_queue; // flush from temporary queue
 | 
					    QueueType flush_queue; // flush from temporary queue
 | 
				
			||||||
    {
 | 
					    {
 | 
				
			||||||
@@ -538,8 +542,8 @@ void BufferedProducer<BufferType>::async_flush() {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::flush(bool preserve_order) {
 | 
					void BufferedProducer<BufferType, Allocator>::flush(bool preserve_order) {
 | 
				
			||||||
    if (preserve_order) {
 | 
					    if (preserve_order) {
 | 
				
			||||||
        CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
					        CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
				
			||||||
        QueueType flush_queue; // flush from temporary queue
 | 
					        QueueType flush_queue; // flush from temporary queue
 | 
				
			||||||
@@ -558,8 +562,8 @@ void BufferedProducer<BufferType>::flush(bool preserve_order) {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
bool BufferedProducer<BufferType>::flush(std::chrono::milliseconds timeout,
 | 
					bool BufferedProducer<BufferType, Allocator>::flush(std::chrono::milliseconds timeout,
 | 
				
			||||||
                                         bool preserve_order) {
 | 
					                                         bool preserve_order) {
 | 
				
			||||||
    if (preserve_order) {
 | 
					    if (preserve_order) {
 | 
				
			||||||
        CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
					        CounterGuard<size_t> counter_guard(flushes_in_progress_);
 | 
				
			||||||
@@ -582,8 +586,8 @@ bool BufferedProducer<BufferType>::flush(std::chrono::milliseconds timeout,
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::wait_for_acks() {
 | 
					void BufferedProducer<BufferType, Allocator>::wait_for_acks() {
 | 
				
			||||||
    while (pending_acks_ > 0) {
 | 
					    while (pending_acks_ > 0) {
 | 
				
			||||||
        try {
 | 
					        try {
 | 
				
			||||||
            producer_.flush();
 | 
					            producer_.flush();
 | 
				
			||||||
@@ -600,8 +604,8 @@ void BufferedProducer<BufferType>::wait_for_acks() {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
bool BufferedProducer<BufferType>::wait_for_acks(std::chrono::milliseconds timeout) {
 | 
					bool BufferedProducer<BufferType, Allocator>::wait_for_acks(std::chrono::milliseconds timeout) {
 | 
				
			||||||
    auto remaining = timeout;
 | 
					    auto remaining = timeout;
 | 
				
			||||||
    auto start_time = std::chrono::high_resolution_clock::now();
 | 
					    auto start_time = std::chrono::high_resolution_clock::now();
 | 
				
			||||||
    while ((pending_acks_ > 0) && (remaining.count() > 0)) {
 | 
					    while ((pending_acks_ > 0) && (remaining.count() > 0)) {
 | 
				
			||||||
@@ -625,45 +629,45 @@ bool BufferedProducer<BufferType>::wait_for_acks(std::chrono::milliseconds timeo
 | 
				
			|||||||
    return (pending_acks_ == 0);
 | 
					    return (pending_acks_ == 0);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::clear() {
 | 
					void BufferedProducer<BufferType, Allocator>::clear() {
 | 
				
			||||||
    std::lock_guard<std::mutex> lock(mutex_);
 | 
					    std::lock_guard<std::mutex> lock(mutex_);
 | 
				
			||||||
    QueueType tmp;
 | 
					    QueueType tmp;
 | 
				
			||||||
    std::swap(tmp, messages_);
 | 
					    std::swap(tmp, messages_);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_buffer_size() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_buffer_size() const {
 | 
				
			||||||
    return messages_.size();
 | 
					    return messages_.size();
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_max_buffer_size(ssize_t max_buffer_size) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_max_buffer_size(ssize_t max_buffer_size) {
 | 
				
			||||||
    if (max_buffer_size < -1) {
 | 
					    if (max_buffer_size < -1) {
 | 
				
			||||||
        throw Exception("Invalid buffer size.");
 | 
					        throw Exception("Invalid buffer size.");
 | 
				
			||||||
    }
 | 
					    }
 | 
				
			||||||
    max_buffer_size_ = max_buffer_size;
 | 
					    max_buffer_size_ = max_buffer_size;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
ssize_t BufferedProducer<BufferType>::get_max_buffer_size() const {
 | 
					ssize_t BufferedProducer<BufferType, Allocator>::get_max_buffer_size() const {
 | 
				
			||||||
    return max_buffer_size_;
 | 
					    return max_buffer_size_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_flush_method(FlushMethod method) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_flush_method(FlushMethod method) {
 | 
				
			||||||
    flush_method_ = method;
 | 
					    flush_method_ = method;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
typename BufferedProducer<BufferType>::FlushMethod
 | 
					typename BufferedProducer<BufferType, Allocator>::FlushMethod
 | 
				
			||||||
BufferedProducer<BufferType>::get_flush_method() const {
 | 
					BufferedProducer<BufferType, Allocator>::get_flush_method() const {
 | 
				
			||||||
    return flush_method_;
 | 
					    return flush_method_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
template <typename BuilderType>
 | 
					template <typename BuilderType>
 | 
				
			||||||
void BufferedProducer<BufferType>::do_add_message(BuilderType&& builder,
 | 
					void BufferedProducer<BufferType, Allocator>::do_add_message(BuilderType&& builder,
 | 
				
			||||||
                                                             MessagePriority priority,
 | 
					                                                             MessagePriority priority,
 | 
				
			||||||
                                                             bool do_flush) {
 | 
					                                                             bool do_flush) {
 | 
				
			||||||
    {
 | 
					    {
 | 
				
			||||||
@@ -685,73 +689,73 @@ void BufferedProducer<BufferType>::do_add_message(BuilderType&& builder,
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
Producer& BufferedProducer<BufferType>::get_producer() {
 | 
					Producer& BufferedProducer<BufferType, Allocator>::get_producer() {
 | 
				
			||||||
    return producer_;
 | 
					    return producer_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
const Producer& BufferedProducer<BufferType>::get_producer() const {
 | 
					const Producer& BufferedProducer<BufferType, Allocator>::get_producer() const {
 | 
				
			||||||
    return producer_;
 | 
					    return producer_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_pending_acks() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_pending_acks() const {
 | 
				
			||||||
    return pending_acks_;
 | 
					    return pending_acks_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_total_messages_produced() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_total_messages_produced() const {
 | 
				
			||||||
    return total_messages_produced_;
 | 
					    return total_messages_produced_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_total_messages_dropped() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_total_messages_dropped() const {
 | 
				
			||||||
    return total_messages_dropped_;
 | 
					    return total_messages_dropped_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_flushes_in_progress() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_flushes_in_progress() const {
 | 
				
			||||||
    return flushes_in_progress_;
 | 
					    return flushes_in_progress_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_max_number_retries(size_t max_number_retries) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_max_number_retries(size_t max_number_retries) {
 | 
				
			||||||
    if (!has_internal_data_ && (max_number_retries > 0)) {
 | 
					    if (!has_internal_data_ && (max_number_retries > 0)) {
 | 
				
			||||||
        has_internal_data_ = true; //enable once
 | 
					        has_internal_data_ = true; //enable once
 | 
				
			||||||
    }
 | 
					    }
 | 
				
			||||||
    max_number_retries_ = max_number_retries;
 | 
					    max_number_retries_ = max_number_retries;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
size_t BufferedProducer<BufferType>::get_max_number_retries() const {
 | 
					size_t BufferedProducer<BufferType, Allocator>::get_max_number_retries() const {
 | 
				
			||||||
    return max_number_retries_;
 | 
					    return max_number_retries_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
typename BufferedProducer<BufferType>::Builder
 | 
					typename BufferedProducer<BufferType, Allocator>::Builder
 | 
				
			||||||
BufferedProducer<BufferType>::make_builder(std::string topic) {
 | 
					BufferedProducer<BufferType, Allocator>::make_builder(std::string topic) {
 | 
				
			||||||
    return Builder(std::move(topic));
 | 
					    return Builder(std::move(topic));
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_produce_failure_callback(ProduceFailureCallback callback) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_produce_failure_callback(ProduceFailureCallback callback) {
 | 
				
			||||||
    produce_failure_callback_ = std::move(callback);
 | 
					    produce_failure_callback_ = std::move(callback);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_produce_success_callback(ProduceSuccessCallback callback) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_produce_success_callback(ProduceSuccessCallback callback) {
 | 
				
			||||||
    produce_success_callback_ = std::move(callback);
 | 
					    produce_success_callback_ = std::move(callback);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::set_flush_failure_callback(FlushFailureCallback callback) {
 | 
					void BufferedProducer<BufferType, Allocator>::set_flush_failure_callback(FlushFailureCallback callback) {
 | 
				
			||||||
    flush_failure_callback_ = std::move(callback);
 | 
					    flush_failure_callback_ = std::move(callback);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
template <typename BuilderType>
 | 
					template <typename BuilderType>
 | 
				
			||||||
void BufferedProducer<BufferType>::produce_message(BuilderType&& builder) {
 | 
					void BufferedProducer<BufferType, Allocator>::produce_message(BuilderType&& builder) {
 | 
				
			||||||
    using builder_type = typename std::decay<BuilderType>::type;
 | 
					    using builder_type = typename std::decay<BuilderType>::type;
 | 
				
			||||||
    while (true) {
 | 
					    while (true) {
 | 
				
			||||||
        try {
 | 
					        try {
 | 
				
			||||||
@@ -774,9 +778,9 @@ void BufferedProducer<BufferType>::produce_message(BuilderType&& builder) {
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
template <typename BuilderType>
 | 
					template <typename BuilderType>
 | 
				
			||||||
void BufferedProducer<BufferType>::async_produce(BuilderType&& builder, bool throw_on_error) {
 | 
					void BufferedProducer<BufferType, Allocator>::async_produce(BuilderType&& builder, bool throw_on_error) {
 | 
				
			||||||
    try {
 | 
					    try {
 | 
				
			||||||
        TestParameters* test_params = get_test_parameters();
 | 
					        TestParameters* test_params = get_test_parameters();
 | 
				
			||||||
        if (test_params && test_params->force_produce_error_) {
 | 
					        if (test_params && test_params->force_produce_error_) {
 | 
				
			||||||
@@ -802,16 +806,16 @@ void BufferedProducer<BufferType>::async_produce(BuilderType&& builder, bool thr
 | 
				
			|||||||
    }
 | 
					    }
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
Configuration BufferedProducer<BufferType>::prepare_configuration(Configuration config) {
 | 
					Configuration BufferedProducer<BufferType, Allocator>::prepare_configuration(Configuration config) {
 | 
				
			||||||
    using std::placeholders::_2;
 | 
					    using std::placeholders::_2;
 | 
				
			||||||
    auto callback = std::bind(&BufferedProducer<BufferType>::on_delivery_report, this, _2);
 | 
					    auto callback = std::bind(&BufferedProducer<BufferType, Allocator>::on_delivery_report, this, _2);
 | 
				
			||||||
    config.set_delivery_report_callback(std::move(callback));
 | 
					    config.set_delivery_report_callback(std::move(callback));
 | 
				
			||||||
    return config;
 | 
					    return config;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename BufferType>
 | 
					template <typename BufferType, typename Allocator>
 | 
				
			||||||
void BufferedProducer<BufferType>::on_delivery_report(const Message& message) {
 | 
					void BufferedProducer<BufferType, Allocator>::on_delivery_report(const Message& message) {
 | 
				
			||||||
    //Get tracker data
 | 
					    //Get tracker data
 | 
				
			||||||
    TestParameters* test_params = get_test_parameters();
 | 
					    TestParameters* test_params = get_test_parameters();
 | 
				
			||||||
    TrackerPtr tracker = has_internal_data_ ?
 | 
					    TrackerPtr tracker = has_internal_data_ ?
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -108,7 +108,7 @@ struct PollInterface {
 | 
				
			|||||||
     * otherwise the broker will think this consumer is down and will trigger a rebalance
 | 
					     * otherwise the broker will think this consumer is down and will trigger a rebalance
 | 
				
			||||||
     * (if using dynamic subscription)
 | 
					     * (if using dynamic subscription)
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    virtual MessageList poll_batch(size_t max_batch_size) = 0;
 | 
					    virtual std::vector<Message> poll_batch(size_t max_batch_size) = 0;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \brief Polls all assigned partitions for a batch of new messages in round-robin fashion
 | 
					     * \brief Polls all assigned partitions for a batch of new messages in round-robin fashion
 | 
				
			||||||
@@ -122,7 +122,7 @@ struct PollInterface {
 | 
				
			|||||||
     *
 | 
					     *
 | 
				
			||||||
     * \return A list of messages
 | 
					     * \return A list of messages
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    virtual MessageList poll_batch(size_t max_batch_size, std::chrono::milliseconds timeout) = 0;
 | 
					    virtual std::vector<Message> poll_batch(size_t max_batch_size, std::chrono::milliseconds timeout) = 0;
 | 
				
			||||||
};
 | 
					};
 | 
				
			||||||
 | 
					
 | 
				
			||||||
} //cppkafka
 | 
					} //cppkafka
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -102,12 +102,19 @@ public:
 | 
				
			|||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \sa PollInterface::poll_batch
 | 
					     * \sa PollInterface::poll_batch
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size) override;
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                               const Allocator& alloc);
 | 
				
			||||||
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size) override;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    /**
 | 
					    /**
 | 
				
			||||||
     * \sa PollInterface::poll_batch
 | 
					     * \sa PollInterface::poll_batch
 | 
				
			||||||
     */
 | 
					     */
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size,
 | 
					    template <typename Allocator>
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                               std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                               const Allocator& alloc);
 | 
				
			||||||
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size,
 | 
				
			||||||
                                    std::chrono::milliseconds timeout) override;
 | 
					                                    std::chrono::milliseconds timeout) override;
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
protected:
 | 
					protected:
 | 
				
			||||||
@@ -119,10 +126,12 @@ protected:
 | 
				
			|||||||
    QueueData& get_next_queue();
 | 
					    QueueData& get_next_queue();
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
private:
 | 
					private:
 | 
				
			||||||
 | 
					    template <typename Allocator>
 | 
				
			||||||
    void consume_batch(Queue& queue,
 | 
					    void consume_batch(Queue& queue,
 | 
				
			||||||
                       MessageList& messages,
 | 
					                       std::vector<Message, Allocator>& messages,
 | 
				
			||||||
                       ssize_t& count,
 | 
					                       ssize_t& count,
 | 
				
			||||||
                       std::chrono::milliseconds timeout);
 | 
					                       std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                       const Allocator& alloc);
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
    void restore_forwarding();
 | 
					    void restore_forwarding();
 | 
				
			||||||
    
 | 
					    
 | 
				
			||||||
@@ -130,6 +139,53 @@ private:
 | 
				
			|||||||
    QueueMap::iterator  queue_iter_;
 | 
					    QueueMap::iterator  queue_iter_;
 | 
				
			||||||
};
 | 
					};
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// Implementations
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> RoundRobinPollStrategy::poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                                   const Allocator& alloc) {
 | 
				
			||||||
 | 
					    return poll_batch(max_batch_size, get_consumer().get_timeout(), alloc);
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					std::vector<Message, Allocator> RoundRobinPollStrategy::poll_batch(size_t max_batch_size,
 | 
				
			||||||
 | 
					                                                                   std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                                                   const Allocator& alloc) {
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> messages(alloc);
 | 
				
			||||||
 | 
					    ssize_t count = max_batch_size;
 | 
				
			||||||
 | 
					    
 | 
				
			||||||
 | 
					    // batch from the group event queue first (non-blocking)
 | 
				
			||||||
 | 
					    consume_batch(get_consumer_queue().queue, messages, count, std::chrono::milliseconds(0), alloc);
 | 
				
			||||||
 | 
					    size_t num_queues = get_partition_queues().size();
 | 
				
			||||||
 | 
					    while ((count > 0) && (num_queues--)) {
 | 
				
			||||||
 | 
					        // batch from the next partition (non-blocking)
 | 
				
			||||||
 | 
					        consume_batch(get_next_queue().queue, messages, count, std::chrono::milliseconds(0), alloc);
 | 
				
			||||||
 | 
					    }
 | 
				
			||||||
 | 
					    // we still have space left in the buffer
 | 
				
			||||||
 | 
					    if (count > 0) {
 | 
				
			||||||
 | 
					        // wait on the event queue until timeout
 | 
				
			||||||
 | 
					        consume_batch(get_consumer_queue().queue, messages, count, timeout, alloc);
 | 
				
			||||||
 | 
					    }
 | 
				
			||||||
 | 
					    return messages;
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					template <typename Allocator>
 | 
				
			||||||
 | 
					void RoundRobinPollStrategy::consume_batch(Queue& queue,
 | 
				
			||||||
 | 
					                                           std::vector<Message, Allocator>& messages,
 | 
				
			||||||
 | 
					                                           ssize_t& count,
 | 
				
			||||||
 | 
					                                           std::chrono::milliseconds timeout,
 | 
				
			||||||
 | 
					                                           const Allocator& alloc) {
 | 
				
			||||||
 | 
					    std::vector<Message, Allocator> queue_messages = queue.consume_batch(count, timeout, alloc);
 | 
				
			||||||
 | 
					    if (queue_messages.empty()) {
 | 
				
			||||||
 | 
					        return;
 | 
				
			||||||
 | 
					    }
 | 
				
			||||||
 | 
					    // concatenate both lists
 | 
				
			||||||
 | 
					    messages.insert(messages.end(),
 | 
				
			||||||
 | 
					                    make_move_iterator(queue_messages.begin()),
 | 
				
			||||||
 | 
					                    make_move_iterator(queue_messages.end()));
 | 
				
			||||||
 | 
					    // reduce total batch count
 | 
				
			||||||
 | 
					    count -= queue_messages.size();
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
} //cppkafka
 | 
					} //cppkafka
 | 
				
			||||||
 | 
					
 | 
				
			||||||
#endif //CPPKAFKA_ROUNDROBIN_POLL_STRATEGY_H
 | 
					#endif //CPPKAFKA_ROUNDROBIN_POLL_STRATEGY_H
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -44,12 +44,13 @@ using std::ostringstream;
 | 
				
			|||||||
using std::chrono::milliseconds;
 | 
					using std::chrono::milliseconds;
 | 
				
			||||||
using std::toupper;
 | 
					using std::toupper;
 | 
				
			||||||
using std::equal;
 | 
					using std::equal;
 | 
				
			||||||
 | 
					using std::allocator;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
namespace cppkafka {
 | 
					namespace cppkafka {
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// See: https://github.com/edenhill/librdkafka/issues/1792
 | 
					// See: https://github.com/edenhill/librdkafka/issues/1792
 | 
				
			||||||
const int rd_kafka_queue_refcount_bug_version = 0x000b0500;
 | 
					const int rd_kafka_queue_refcount_bug_version = 0x000b0500;
 | 
				
			||||||
Queue get_queue(rd_kafka_queue_t* handle) {
 | 
					Queue Consumer::get_queue(rd_kafka_queue_t* handle) {
 | 
				
			||||||
    if (rd_kafka_version() <= rd_kafka_queue_refcount_bug_version) {
 | 
					    if (rd_kafka_version() <= rd_kafka_queue_refcount_bug_version) {
 | 
				
			||||||
        return Queue::make_non_owning(handle);
 | 
					        return Queue::make_non_owning(handle);
 | 
				
			||||||
    }
 | 
					    }
 | 
				
			||||||
@@ -255,22 +256,12 @@ Message Consumer::poll(milliseconds timeout) {
 | 
				
			|||||||
    return rd_kafka_consumer_poll(get_handle(), static_cast<int>(timeout.count()));
 | 
					    return rd_kafka_consumer_poll(get_handle(), static_cast<int>(timeout.count()));
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList Consumer::poll_batch(size_t max_batch_size) {
 | 
					std::vector<Message> Consumer::poll_batch(size_t max_batch_size) {
 | 
				
			||||||
    return poll_batch(max_batch_size, get_timeout());
 | 
					    return poll_batch(max_batch_size, get_timeout(), allocator<Message>());
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList Consumer::poll_batch(size_t max_batch_size, milliseconds timeout) {
 | 
					std::vector<Message> Consumer::poll_batch(size_t max_batch_size, milliseconds timeout) {
 | 
				
			||||||
    vector<rd_kafka_message_t*> raw_messages(max_batch_size);
 | 
					    return poll_batch(max_batch_size, timeout, allocator<Message>());
 | 
				
			||||||
    // Note that this will leak the queue when using rdkafka < 0.11.5 (see get_queue comment)
 | 
					 | 
				
			||||||
    Queue queue(get_queue(rd_kafka_queue_get_consumer(get_handle())));
 | 
					 | 
				
			||||||
    ssize_t result = rd_kafka_consume_batch_queue(queue.get_handle() , timeout.count(), raw_messages.data(),
 | 
					 | 
				
			||||||
                                                  raw_messages.size());
 | 
					 | 
				
			||||||
    if (result == -1) {
 | 
					 | 
				
			||||||
        check_error(rd_kafka_last_error());
 | 
					 | 
				
			||||||
        // on the off-chance that check_error() does not throw an error
 | 
					 | 
				
			||||||
        return MessageList();
 | 
					 | 
				
			||||||
    }
 | 
					 | 
				
			||||||
    return MessageList(raw_messages.begin(), raw_messages.begin() + result);
 | 
					 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
Queue Consumer::get_main_queue() const {
 | 
					Queue Consumer::get_main_queue() const {
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -32,6 +32,7 @@
 | 
				
			|||||||
using std::vector;
 | 
					using std::vector;
 | 
				
			||||||
using std::exception;
 | 
					using std::exception;
 | 
				
			||||||
using std::chrono::milliseconds;
 | 
					using std::chrono::milliseconds;
 | 
				
			||||||
 | 
					using std::allocator;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
namespace cppkafka {
 | 
					namespace cppkafka {
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -94,25 +95,13 @@ Message Queue::consume(milliseconds timeout) const {
 | 
				
			|||||||
    return Message(rd_kafka_consume_queue(handle_.get(), static_cast<int>(timeout.count())));
 | 
					    return Message(rd_kafka_consume_queue(handle_.get(), static_cast<int>(timeout.count())));
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList Queue::consume_batch(size_t max_batch_size) const {
 | 
					std::vector<Message> Queue::consume_batch(size_t max_batch_size) const {
 | 
				
			||||||
    return consume_batch(max_batch_size, timeout_ms_);
 | 
					    return consume_batch(max_batch_size, timeout_ms_, allocator<Message>());
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList Queue::consume_batch(size_t max_batch_size, milliseconds timeout) const {
 | 
					std::vector<Message> Queue::consume_batch(size_t max_batch_size,
 | 
				
			||||||
    vector<rd_kafka_message_t*> raw_messages(max_batch_size);
 | 
					                                          milliseconds timeout) const {
 | 
				
			||||||
    ssize_t result = rd_kafka_consume_batch_queue(handle_.get(),
 | 
					    return consume_batch(max_batch_size, timeout, allocator<Message>());
 | 
				
			||||||
                                                  static_cast<int>(timeout.count()),
 | 
					 | 
				
			||||||
                                                  raw_messages.data(),
 | 
					 | 
				
			||||||
                                                  raw_messages.size());
 | 
					 | 
				
			||||||
    if (result == -1) {
 | 
					 | 
				
			||||||
        rd_kafka_resp_err_t error = rd_kafka_last_error();
 | 
					 | 
				
			||||||
        if (error != RD_KAFKA_RESP_ERR_NO_ERROR) {
 | 
					 | 
				
			||||||
            throw QueueException(error);
 | 
					 | 
				
			||||||
        }
 | 
					 | 
				
			||||||
        return MessageList();
 | 
					 | 
				
			||||||
    }
 | 
					 | 
				
			||||||
    // Build message list
 | 
					 | 
				
			||||||
    return MessageList(raw_messages.begin(), raw_messages.begin() + result);
 | 
					 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
} //cppkafka
 | 
					} //cppkafka
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -32,6 +32,7 @@
 | 
				
			|||||||
using std::string;
 | 
					using std::string;
 | 
				
			||||||
using std::chrono::milliseconds;
 | 
					using std::chrono::milliseconds;
 | 
				
			||||||
using std::make_move_iterator;
 | 
					using std::make_move_iterator;
 | 
				
			||||||
 | 
					using std::allocator;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
namespace cppkafka {
 | 
					namespace cppkafka {
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -67,46 +68,15 @@ Message RoundRobinPollStrategy::poll(milliseconds timeout) {
 | 
				
			|||||||
    return get_consumer_queue().queue.consume(timeout);
 | 
					    return get_consumer_queue().queue.consume(timeout);
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList RoundRobinPollStrategy::poll_batch(size_t max_batch_size) {
 | 
					std::vector<Message> RoundRobinPollStrategy::poll_batch(size_t max_batch_size) {
 | 
				
			||||||
    return poll_batch(max_batch_size, get_consumer().get_timeout());
 | 
					    return poll_batch(max_batch_size, get_consumer().get_timeout(), allocator<Message>());
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
MessageList RoundRobinPollStrategy::poll_batch(size_t max_batch_size, milliseconds timeout) {
 | 
					std::vector<Message> RoundRobinPollStrategy::poll_batch(size_t max_batch_size,
 | 
				
			||||||
    MessageList messages;
 | 
					 | 
				
			||||||
    ssize_t count = max_batch_size;
 | 
					 | 
				
			||||||
    
 | 
					 | 
				
			||||||
    // batch from the group event queue first (non-blocking)
 | 
					 | 
				
			||||||
    consume_batch(get_consumer_queue().queue, messages, count, milliseconds(0));
 | 
					 | 
				
			||||||
    size_t num_queues = get_partition_queues().size();
 | 
					 | 
				
			||||||
    while ((count > 0) && (num_queues--)) {
 | 
					 | 
				
			||||||
        // batch from the next partition (non-blocking)
 | 
					 | 
				
			||||||
        consume_batch(get_next_queue().queue, messages, count, milliseconds(0));
 | 
					 | 
				
			||||||
    }
 | 
					 | 
				
			||||||
    // we still have space left in the buffer
 | 
					 | 
				
			||||||
    if (count > 0) {
 | 
					 | 
				
			||||||
        // wait on the event queue until timeout
 | 
					 | 
				
			||||||
        consume_batch(get_consumer_queue().queue, messages, count, timeout);
 | 
					 | 
				
			||||||
    }
 | 
					 | 
				
			||||||
    return messages;
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
void RoundRobinPollStrategy::consume_batch(Queue& queue,
 | 
					 | 
				
			||||||
                                           MessageList& messages,
 | 
					 | 
				
			||||||
                                           ssize_t& count,
 | 
					 | 
				
			||||||
                                                        milliseconds timeout) {
 | 
					                                                        milliseconds timeout) {
 | 
				
			||||||
    MessageList queue_messages = queue.consume_batch(count, timeout);
 | 
					    return poll_batch(max_batch_size, timeout, allocator<Message>());
 | 
				
			||||||
    if (queue_messages.empty()) {
 | 
					 | 
				
			||||||
        return;
 | 
					 | 
				
			||||||
    }
 | 
					 | 
				
			||||||
    // concatenate both lists
 | 
					 | 
				
			||||||
    messages.insert(messages.end(),
 | 
					 | 
				
			||||||
                    make_move_iterator(queue_messages.begin()),
 | 
					 | 
				
			||||||
                    make_move_iterator(queue_messages.end()));
 | 
					 | 
				
			||||||
    // reduce total batch count
 | 
					 | 
				
			||||||
    count -= queue_messages.size();
 | 
					 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					 | 
				
			||||||
void RoundRobinPollStrategy::restore_forwarding() {
 | 
					void RoundRobinPollStrategy::restore_forwarding() {
 | 
				
			||||||
    // forward all partition queues
 | 
					    // forward all partition queues
 | 
				
			||||||
    for (const auto& toppar : get_partition_queues()) {
 | 
					    for (const auto& toppar : get_partition_queues()) {
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -48,8 +48,8 @@ public:
 | 
				
			|||||||
    void delete_polling_strategy();
 | 
					    void delete_polling_strategy();
 | 
				
			||||||
    Message poll();
 | 
					    Message poll();
 | 
				
			||||||
    Message poll(std::chrono::milliseconds timeout);
 | 
					    Message poll(std::chrono::milliseconds timeout);
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size);
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size);
 | 
				
			||||||
    MessageList poll_batch(size_t max_batch_size,
 | 
					    std::vector<Message> poll_batch(size_t max_batch_size,
 | 
				
			||||||
                                    std::chrono::milliseconds timeout);
 | 
					                                    std::chrono::milliseconds timeout);
 | 
				
			||||||
    void set_timeout(std::chrono::milliseconds timeout);
 | 
					    void set_timeout(std::chrono::milliseconds timeout);
 | 
				
			||||||
    std::chrono::milliseconds get_timeout();
 | 
					    std::chrono::milliseconds get_timeout();
 | 
				
			||||||
 
 | 
				
			|||||||
@@ -19,7 +19,6 @@ using cppkafka::Consumer;
 | 
				
			|||||||
using cppkafka::BasicConsumerDispatcher;
 | 
					using cppkafka::BasicConsumerDispatcher;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
using cppkafka::Message;
 | 
					using cppkafka::Message;
 | 
				
			||||||
using cppkafka::MessageList;
 | 
					 | 
				
			||||||
using cppkafka::TopicPartition;
 | 
					using cppkafka::TopicPartition;
 | 
				
			||||||
 | 
					
 | 
				
			||||||
//==================================================================================
 | 
					//==================================================================================
 | 
				
			||||||
@@ -89,7 +88,7 @@ BasicConsumerRunner<ConsumerType>::~BasicConsumerRunner() {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
template <typename ConsumerType>
 | 
					template <typename ConsumerType>
 | 
				
			||||||
const MessageList& BasicConsumerRunner<ConsumerType>::get_messages() const {
 | 
					const std::vector<Message>& BasicConsumerRunner<ConsumerType>::get_messages() const {
 | 
				
			||||||
    return messages_;
 | 
					    return messages_;
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -135,7 +134,7 @@ Message PollStrategyAdapter::poll(milliseconds timeout) {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
inline
 | 
					inline
 | 
				
			||||||
MessageList PollStrategyAdapter::poll_batch(size_t max_batch_size) {
 | 
					std::vector<Message> PollStrategyAdapter::poll_batch(size_t max_batch_size) {
 | 
				
			||||||
    if (strategy_) {
 | 
					    if (strategy_) {
 | 
				
			||||||
        return strategy_->poll_batch(max_batch_size);
 | 
					        return strategy_->poll_batch(max_batch_size);
 | 
				
			||||||
    }
 | 
					    }
 | 
				
			||||||
@@ -143,7 +142,7 @@ MessageList PollStrategyAdapter::poll_batch(size_t max_batch_size) {
 | 
				
			|||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
inline
 | 
					inline
 | 
				
			||||||
MessageList PollStrategyAdapter::poll_batch(size_t max_batch_size,
 | 
					std::vector<Message> PollStrategyAdapter::poll_batch(size_t max_batch_size,
 | 
				
			||||||
                                                     milliseconds timeout) {
 | 
					                                                     milliseconds timeout) {
 | 
				
			||||||
    if (strategy_) {
 | 
					    if (strategy_) {
 | 
				
			||||||
        return strategy_->poll_batch(max_batch_size, timeout);
 | 
					        return strategy_->poll_batch(max_batch_size, timeout);
 | 
				
			||||||
 
 | 
				
			|||||||
		Reference in New Issue
	
	Block a user