2020-05-20 09:40:49 +00:00
|
|
|
#include <utility>
|
2020-05-26 17:34:57 +00:00
|
|
|
#include <chrono>
|
|
|
|
#include <thread>
|
2020-05-31 08:39:22 +00:00
|
|
|
#include <mutex>
|
|
|
|
#include <atomic>
|
|
|
|
#include <memory>
|
2020-05-20 09:40:49 +00:00
|
|
|
#include <Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h>
|
|
|
|
#include <Storages/RabbitMQ/RabbitMQHandler.h>
|
2020-06-13 21:37:37 +00:00
|
|
|
#include <boost/algorithm/string/split.hpp>
|
2020-05-20 09:40:49 +00:00
|
|
|
#include <common/logger_useful.h>
|
2020-06-07 11:14:05 +00:00
|
|
|
#include "Poco/Timer.h"
|
2020-05-20 09:40:49 +00:00
|
|
|
#include <amqpcpp.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-08-15 14:38:29 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer(
|
2020-05-29 16:04:44 +00:00
|
|
|
ChannelPtr consumer_channel_,
|
2020-07-21 15:47:39 +00:00
|
|
|
ChannelPtr setup_channel_,
|
2020-06-30 01:48:11 +00:00
|
|
|
HandlerPtr event_handler_,
|
2020-05-20 09:40:49 +00:00
|
|
|
const String & exchange_name_,
|
2020-08-15 06:50:53 +00:00
|
|
|
size_t channel_id_base_,
|
|
|
|
const String & channel_base_,
|
2020-07-23 11:45:01 +00:00
|
|
|
const String & queue_base_,
|
2020-05-20 09:40:49 +00:00
|
|
|
Poco::Logger * log_,
|
|
|
|
char row_delimiter_,
|
2020-07-20 06:21:18 +00:00
|
|
|
bool hash_exchange_,
|
2020-07-02 16:44:04 +00:00
|
|
|
size_t num_queues_,
|
2020-07-24 12:33:07 +00:00
|
|
|
const String & deadletter_exchange_,
|
2020-09-01 14:11:34 +00:00
|
|
|
uint32_t queue_size_,
|
2020-05-20 09:40:49 +00:00
|
|
|
const std::atomic<bool> & stopped_)
|
|
|
|
: ReadBuffer(nullptr, 0)
|
2020-05-29 16:04:44 +00:00
|
|
|
, consumer_channel(std::move(consumer_channel_))
|
2020-07-21 15:47:39 +00:00
|
|
|
, setup_channel(setup_channel_)
|
2020-06-30 01:48:11 +00:00
|
|
|
, event_handler(event_handler_)
|
2020-05-20 09:40:49 +00:00
|
|
|
, exchange_name(exchange_name_)
|
2020-08-15 06:50:53 +00:00
|
|
|
, channel_base(channel_base_)
|
|
|
|
, channel_id_base(channel_id_base_)
|
2020-07-23 11:45:01 +00:00
|
|
|
, queue_base(queue_base_)
|
2020-07-20 06:21:18 +00:00
|
|
|
, hash_exchange(hash_exchange_)
|
2020-05-20 09:40:49 +00:00
|
|
|
, num_queues(num_queues_)
|
2020-08-15 06:50:53 +00:00
|
|
|
, deadletter_exchange(deadletter_exchange_)
|
2020-07-13 01:11:35 +00:00
|
|
|
, log(log_)
|
|
|
|
, row_delimiter(row_delimiter_)
|
2020-09-01 14:11:34 +00:00
|
|
|
, queue_size(queue_size_)
|
2020-05-20 09:40:49 +00:00
|
|
|
, stopped(stopped_)
|
2020-09-01 14:11:34 +00:00
|
|
|
, received(queue_size)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
for (size_t queue_id = 0; queue_id < num_queues; ++queue_id)
|
2020-07-29 19:45:18 +00:00
|
|
|
bindQueue(queue_id);
|
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
setupChannel();
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer()
|
|
|
|
{
|
|
|
|
BufferBase::set(nullptr, 0, 0);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-07-29 19:45:18 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::bindQueue(size_t queue_id)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-08-15 06:50:53 +00:00
|
|
|
std::atomic<bool> binding_created = false;
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-08-01 12:52:00 +00:00
|
|
|
auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-08-01 12:52:00 +00:00
|
|
|
queues.emplace_back(queue_name);
|
|
|
|
LOG_DEBUG(log, "Queue {} is declared", queue_name);
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-07-23 11:45:01 +00:00
|
|
|
if (msgcount)
|
2020-08-01 12:52:00 +00:00
|
|
|
LOG_TRACE(log, "Queue {} is non-empty. Non-consumed messaged will also be delivered", queue_name);
|
2020-07-23 11:45:01 +00:00
|
|
|
|
2020-08-06 13:33:46 +00:00
|
|
|
/* Here we bind either to sharding exchange (consistent-hash) or to bridge exchange (fanout). All bindings to routing keys are
|
|
|
|
* done between client's exchange and local bridge exchange. Binding key must be a string integer in case of hash exchange, for
|
2020-08-28 08:52:02 +00:00
|
|
|
* fanout exchange it can be arbitrary
|
2020-08-06 13:33:46 +00:00
|
|
|
*/
|
2020-08-15 06:50:53 +00:00
|
|
|
setup_channel->bindQueue(exchange_name, queue_name, std::to_string(channel_id_base))
|
2020-07-29 19:45:18 +00:00
|
|
|
.onSuccess([&]
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-08-15 06:50:53 +00:00
|
|
|
binding_created = true;
|
2020-07-29 19:45:18 +00:00
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
2020-07-20 06:21:18 +00:00
|
|
|
{
|
2020-08-15 06:50:53 +00:00
|
|
|
throw Exception("Failed to create queue binding. Reason: " + std::string(message), ErrorCodes::LOGICAL_ERROR);
|
2020-07-29 19:45:18 +00:00
|
|
|
});
|
2020-07-23 11:45:01 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
auto error_callback([&](const char * message)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-09-01 14:11:34 +00:00
|
|
|
/* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a
|
|
|
|
* given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different
|
|
|
|
* max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously
|
|
|
|
* declared queues via any of the various cli tools.
|
|
|
|
*/
|
|
|
|
throw Exception("Failed to declare queue. Probably queue settings are conflicting: max_block_size, deadletter_exchange. Attempt \
|
|
|
|
specifying differently those settings or use a different queue_base or manually delete previously declared queues, \
|
|
|
|
which were declared with the same names. ERROR reason: "
|
|
|
|
+ std::string(message), ErrorCodes::LOGICAL_ERROR);
|
2020-05-20 09:40:49 +00:00
|
|
|
});
|
|
|
|
|
2020-07-24 12:33:07 +00:00
|
|
|
AMQP::Table queue_settings;
|
2020-09-01 14:11:34 +00:00
|
|
|
|
|
|
|
queue_settings["x-max-length"] = queue_size;
|
|
|
|
queue_settings["x-overflow"] = "reject-publish";
|
|
|
|
|
2020-07-24 12:33:07 +00:00
|
|
|
if (!deadletter_exchange.empty())
|
|
|
|
queue_settings["x-dead-letter-exchange"] = deadletter_exchange;
|
|
|
|
|
2020-08-06 13:33:46 +00:00
|
|
|
/* The first option not just simplifies queue_name, but also implements the possibility to be able to resume reading from one
|
2020-08-28 08:52:02 +00:00
|
|
|
* specific queue when its name is specified in queue_base setting
|
2020-08-06 13:33:46 +00:00
|
|
|
*/
|
2020-08-26 08:54:29 +00:00
|
|
|
const String queue_name = !hash_exchange ? queue_base : std::to_string(channel_id_base) + "_" + std::to_string(queue_id) + "_" + queue_base;
|
2020-08-06 13:33:46 +00:00
|
|
|
setup_channel->declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback);
|
2020-07-23 11:45:01 +00:00
|
|
|
|
2020-08-15 06:50:53 +00:00
|
|
|
while (!binding_created)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-07-02 16:44:04 +00:00
|
|
|
iterateEventLoop();
|
2020-06-08 01:11:48 +00:00
|
|
|
}
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-07-29 19:45:18 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::subscribe()
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-07-29 19:45:18 +00:00
|
|
|
for (const auto & queue_name : queues)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-07-29 19:45:18 +00:00
|
|
|
consumer_channel->consume(queue_name)
|
2020-08-15 06:50:53 +00:00
|
|
|
.onSuccess([&](const std::string & /* consumer_tag */)
|
2020-07-29 19:45:18 +00:00
|
|
|
{
|
2020-08-15 06:50:53 +00:00
|
|
|
LOG_TRACE(log, "Consumer on channel {} is subscribed to queue {}", channel_id, queue_name);
|
2020-08-28 08:52:02 +00:00
|
|
|
|
|
|
|
if (++subscribed == queues.size())
|
|
|
|
wait_subscription.store(false);
|
2020-07-29 19:45:18 +00:00
|
|
|
})
|
|
|
|
.onReceived([&](const AMQP::Message & message, uint64_t delivery_tag, bool redelivered)
|
2020-07-24 12:33:07 +00:00
|
|
|
{
|
2020-07-29 19:45:18 +00:00
|
|
|
if (message.bodySize())
|
|
|
|
{
|
|
|
|
String message_received = std::string(message.body(), message.body() + message.bodySize());
|
|
|
|
if (row_delimiter != '\0')
|
|
|
|
message_received += row_delimiter;
|
|
|
|
|
2020-08-15 14:38:29 +00:00
|
|
|
if (message.hasMessageID())
|
|
|
|
received.push({message_received, message.messageID(), redelivered, AckTracker(delivery_tag, channel_id)});
|
|
|
|
else
|
|
|
|
received.push({message_received, "", redelivered, AckTracker(delivery_tag, channel_id)});
|
2020-07-29 19:45:18 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-08-28 08:52:02 +00:00
|
|
|
/* End up here either if channel ends up in an error state (then there will be resubscription) or consume call error, which
|
|
|
|
* arises from queue settings mismatch or queue level error, which should not happen as noone else is supposed to touch them
|
|
|
|
*/
|
2020-08-15 06:50:53 +00:00
|
|
|
LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message);
|
2020-08-28 08:52:02 +00:00
|
|
|
wait_subscription.store(false);
|
2020-07-29 19:45:18 +00:00
|
|
|
});
|
|
|
|
}
|
2020-06-09 21:52:06 +00:00
|
|
|
}
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-06-08 01:11:48 +00:00
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
bool ReadBufferFromRabbitMQConsumer::ackMessages()
|
2020-07-24 12:33:07 +00:00
|
|
|
{
|
2020-08-15 06:50:53 +00:00
|
|
|
AckTracker record = last_inserted_record;
|
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
/* Do not send ack to server if message's channel is not the same as current running channel because delivery tags are scoped per
|
|
|
|
* channel, so if channel fails, all previous delivery tags become invalid
|
|
|
|
*/
|
|
|
|
if (record.channel_id == channel_id && record.delivery_tag && record.delivery_tag > prev_tag)
|
2020-07-24 12:33:07 +00:00
|
|
|
{
|
2020-08-28 08:52:02 +00:00
|
|
|
/// Commit all received messages with delivery tags from last commited to last inserted
|
|
|
|
if (!consumer_channel->ack(record.delivery_tag, AMQP::multiple))
|
2020-08-31 16:34:16 +00:00
|
|
|
{
|
|
|
|
LOG_ERROR(log, "Failed to commit messages with delivery tags from last commited to {} on channel {}",
|
|
|
|
record.delivery_tag, channel_id);
|
2020-08-28 08:52:02 +00:00
|
|
|
return false;
|
2020-08-31 16:34:16 +00:00
|
|
|
}
|
2020-08-15 06:50:53 +00:00
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
prev_tag = record.delivery_tag;
|
2020-08-31 16:34:16 +00:00
|
|
|
LOG_TRACE(log, "Consumer commited messages with deliveryTags up to {} on channel {}", record.delivery_tag, channel_id);
|
2020-07-24 12:33:07 +00:00
|
|
|
}
|
2020-08-28 08:52:02 +00:00
|
|
|
|
|
|
|
return true;
|
2020-07-24 12:33:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-15 06:50:53 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::updateAckTracker(AckTracker record)
|
|
|
|
{
|
|
|
|
if (record.delivery_tag && channel_error.load())
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (!record.delivery_tag)
|
|
|
|
prev_tag = 0;
|
|
|
|
|
|
|
|
last_inserted_record = record;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::setupChannel()
|
2020-08-15 06:50:53 +00:00
|
|
|
{
|
2020-08-28 08:52:02 +00:00
|
|
|
wait_subscription.store(true);
|
|
|
|
|
2020-08-15 06:50:53 +00:00
|
|
|
consumer_channel->onReady([&]()
|
|
|
|
{
|
2020-08-15 14:38:29 +00:00
|
|
|
/* First number indicates current consumer buffer; second number indicates serial number of created channel for current buffer,
|
|
|
|
* i.e. if channel fails - another one is created and its serial number is incremented; channel_base is to guarantee that
|
2020-08-28 08:52:02 +00:00
|
|
|
* channel_id is unique for each table
|
2020-08-15 14:38:29 +00:00
|
|
|
*/
|
|
|
|
channel_id = std::to_string(channel_id_base) + "_" + std::to_string(channel_id_counter++) + "_" + channel_base;
|
2020-08-15 06:50:53 +00:00
|
|
|
LOG_TRACE(log, "Channel {} is created", channel_id);
|
|
|
|
|
2020-08-28 08:52:02 +00:00
|
|
|
subscribed = 0;
|
2020-08-15 06:50:53 +00:00
|
|
|
subscribe();
|
|
|
|
channel_error.store(false);
|
|
|
|
});
|
2020-08-28 08:52:02 +00:00
|
|
|
|
|
|
|
consumer_channel->onError([&](const char * message)
|
|
|
|
{
|
2020-08-31 16:34:16 +00:00
|
|
|
/// If here, then fatal error occured on the channel and it is not usable anymore, need to close it
|
|
|
|
consumer_channel->close();
|
2020-08-28 08:52:02 +00:00
|
|
|
LOG_ERROR(log, "Channel {} error: {}", channel_id, message);
|
|
|
|
|
|
|
|
channel_error.store(true);
|
|
|
|
wait_subscription.store(false);
|
|
|
|
});
|
2020-08-15 06:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-07-02 16:44:04 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::iterateEventLoop()
|
2020-06-04 06:22:53 +00:00
|
|
|
{
|
2020-07-02 16:44:04 +00:00
|
|
|
event_handler->iterateLoop();
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
bool ReadBufferFromRabbitMQConsumer::nextImpl()
|
|
|
|
{
|
|
|
|
if (stopped || !allowed)
|
|
|
|
return false;
|
|
|
|
|
2020-07-20 10:05:00 +00:00
|
|
|
if (received.tryPop(current))
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-07-20 10:05:00 +00:00
|
|
|
auto * new_position = const_cast<char *>(current.message.data());
|
|
|
|
BufferBase::set(new_position, current.message.size(), 0);
|
2020-06-29 06:33:53 +00:00
|
|
|
allowed = false;
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-06-29 06:33:53 +00:00
|
|
|
return true;
|
|
|
|
}
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-06-29 06:33:53 +00:00
|
|
|
return false;
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|