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>
|
|
|
|
#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-06-08 01:11:48 +00:00
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer(
|
2020-05-29 16:04:44 +00:00
|
|
|
ChannelPtr consumer_channel_,
|
|
|
|
RabbitMQHandler & eventHandler_,
|
2020-05-20 09:40:49 +00:00
|
|
|
const String & exchange_name_,
|
|
|
|
const String & routing_key_,
|
|
|
|
const size_t channel_id_,
|
|
|
|
Poco::Logger * log_,
|
|
|
|
char row_delimiter_,
|
|
|
|
const bool bind_by_id_,
|
|
|
|
const bool hash_exchange_,
|
|
|
|
const size_t num_queues_,
|
|
|
|
const std::atomic<bool> & stopped_)
|
|
|
|
: ReadBuffer(nullptr, 0)
|
2020-05-29 16:04:44 +00:00
|
|
|
, consumer_channel(std::move(consumer_channel_))
|
|
|
|
, eventHandler(eventHandler_)
|
2020-05-20 09:40:49 +00:00
|
|
|
, exchange_name(exchange_name_)
|
|
|
|
, routing_key(routing_key_)
|
|
|
|
, channel_id(channel_id_)
|
|
|
|
, log(log_)
|
|
|
|
, row_delimiter(row_delimiter_)
|
|
|
|
, bind_by_id(bind_by_id_)
|
|
|
|
, hash_exchange(hash_exchange_)
|
|
|
|
, num_queues(num_queues_)
|
|
|
|
, stopped(stopped_)
|
|
|
|
{
|
|
|
|
messages.clear();
|
|
|
|
current = messages.begin();
|
|
|
|
|
|
|
|
/* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added.
|
|
|
|
* By default there is one queue per consumer.
|
|
|
|
*/
|
|
|
|
for (size_t queue_id = 0; queue_id < num_queues; ++queue_id)
|
|
|
|
{
|
|
|
|
initQueueBindings(queue_id);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ReadBufferFromRabbitMQConsumer::~ReadBufferFromRabbitMQConsumer()
|
|
|
|
{
|
2020-05-29 16:04:44 +00:00
|
|
|
consumer_channel->close();
|
2020-05-20 09:40:49 +00:00
|
|
|
|
|
|
|
messages.clear();
|
|
|
|
current = messages.begin();
|
|
|
|
BufferBase::set(nullptr, 0, 0);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReadBufferFromRabbitMQConsumer::initExchange()
|
|
|
|
{
|
|
|
|
/* As there are 5 different types of exchanges and the type should be set as a parameter while publishing the message,
|
|
|
|
* then for uniformity this parameter should always be set as fanout-exchange type. In current implementation, the exchange,
|
|
|
|
* to which messages a published, will be bound to the exchange of the needed type, which will distribute messages according to its type.
|
|
|
|
*/
|
|
|
|
consumer_channel->declareExchange(exchange_name, AMQP::fanout).onError([&](const char * message)
|
|
|
|
{
|
|
|
|
exchange_declared = false;
|
2020-05-26 20:43:20 +00:00
|
|
|
LOG_ERROR(log, "Failed to declare fanout exchange: {}", message);
|
2020-05-20 09:40:49 +00:00
|
|
|
});
|
|
|
|
|
|
|
|
if (hash_exchange)
|
|
|
|
{
|
|
|
|
current_exchange_name = exchange_name + "_hash";
|
2020-05-26 17:34:57 +00:00
|
|
|
consumer_channel->declareExchange(current_exchange_name, AMQP::consistent_hash).onError([&](const char * /* message */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
exchange_declared = false;
|
|
|
|
});
|
|
|
|
|
2020-05-26 17:34:57 +00:00
|
|
|
consumer_channel->bindExchange(exchange_name, current_exchange_name, routing_key).onError([&](const char * /* message */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
exchange_declared = false;
|
|
|
|
});
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
current_exchange_name = exchange_name + "_direct";
|
2020-05-26 17:34:57 +00:00
|
|
|
consumer_channel->declareExchange(current_exchange_name, AMQP::direct).onError([&](const char * /* message */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
exchange_declared = false;
|
|
|
|
});
|
|
|
|
|
2020-05-26 17:34:57 +00:00
|
|
|
consumer_channel->bindExchange(exchange_name, current_exchange_name, routing_key).onError([&](const char * /* message */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
exchange_declared = false;
|
|
|
|
});
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
|
|
|
|
{
|
|
|
|
if (!exchange_declared)
|
|
|
|
{
|
|
|
|
initExchange();
|
|
|
|
exchange_declared = true;
|
|
|
|
}
|
|
|
|
|
2020-05-29 16:04:44 +00:00
|
|
|
std::atomic<bool> bindings_created = false, bindings_error = false;
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2020-06-07 11:14:05 +00:00
|
|
|
consumer_channel->declareQueue(AMQP::exclusive)
|
2020-05-20 09:40:49 +00:00
|
|
|
.onSuccess([&](const std::string & queue_name_, int /* msgcount */, int /* consumercount */)
|
|
|
|
{
|
|
|
|
queues.emplace_back(queue_name_);
|
2020-06-08 01:11:48 +00:00
|
|
|
subscribed_queue[queue_name_] = false;
|
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
String binding_key = routing_key;
|
|
|
|
|
2020-05-26 17:34:57 +00:00
|
|
|
/* Every consumer has at least one unique queue. Bind the queues to exchange based on the consumer_channel_id
|
|
|
|
* in case there is one queue per consumer and bind by queue_id in case there is more than 1 queue per consumer.
|
|
|
|
* (queue_id is based on channel_id)
|
|
|
|
*/
|
|
|
|
if (bind_by_id || hash_exchange)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
if (queues.size() == 1)
|
|
|
|
{
|
2020-05-26 17:34:57 +00:00
|
|
|
binding_key = std::to_string(channel_id);
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-05-26 17:34:57 +00:00
|
|
|
binding_key = std::to_string(channel_id + queue_id);
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
/// Must be done here, cannot be done in readPrefix()
|
|
|
|
subscribe(queues.back());
|
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
LOG_TRACE(log, "Queue " + queue_name_ + " is bound by key " + binding_key);
|
|
|
|
|
|
|
|
consumer_channel->bindQueue(current_exchange_name, queue_name_, binding_key)
|
|
|
|
.onSuccess([&]
|
|
|
|
{
|
2020-05-26 17:34:57 +00:00
|
|
|
bindings_created = true;
|
2020-05-20 09:40:49 +00:00
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
|
|
|
{
|
|
|
|
bindings_error = true;
|
2020-05-26 20:43:20 +00:00
|
|
|
LOG_ERROR(log, "Failed to create queue binding: {}", message);
|
2020-05-20 09:40:49 +00:00
|
|
|
});
|
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
|
|
|
{
|
|
|
|
bindings_error = true;
|
2020-05-26 20:43:20 +00:00
|
|
|
LOG_ERROR(log, "Failed to declare queue on the channel: {}", message);
|
2020-05-20 09:40:49 +00:00
|
|
|
});
|
|
|
|
|
2020-05-26 17:34:57 +00:00
|
|
|
/* Run event loop (which updates local variables in a separate thread) until bindings are created or failed to be created.
|
|
|
|
* It is important at this moment to make sure that queue bindings are created before any publishing can happen because
|
|
|
|
* otherwise messages will be routed nowhere.
|
|
|
|
*/
|
|
|
|
while (!bindings_created && !bindings_error)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-06-09 21:52:06 +00:00
|
|
|
startEventLoop(loop_started);
|
2020-06-08 01:11:48 +00:00
|
|
|
}
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReadBufferFromRabbitMQConsumer::subscribe(const String & queue_name)
|
|
|
|
{
|
2020-06-08 01:11:48 +00:00
|
|
|
if (subscribed_queue[queue_name])
|
|
|
|
return;
|
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
consumer_channel->consume(queue_name, AMQP::noack)
|
2020-05-26 17:34:57 +00:00
|
|
|
.onSuccess([&](const std::string & /* consumer */)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-06-09 21:52:06 +00:00
|
|
|
subscribed_queue[queue_name] = true;
|
2020-06-08 01:11:48 +00:00
|
|
|
++count_subscribed;
|
|
|
|
|
2020-06-07 11:14:05 +00:00
|
|
|
LOG_TRACE(log, "Consumer {} is subscribed to queue {}", channel_id, queue_name);
|
2020-05-20 09:40:49 +00:00
|
|
|
})
|
|
|
|
.onReceived([&](const AMQP::Message & message, uint64_t /* deliveryTag */, bool /* redelivered */)
|
|
|
|
{
|
|
|
|
size_t message_size = message.bodySize();
|
|
|
|
if (message_size && message.body() != nullptr)
|
|
|
|
{
|
|
|
|
String message_received = std::string(message.body(), message.body() + message_size);
|
|
|
|
|
|
|
|
if (row_delimiter != '\0')
|
2020-06-07 11:14:05 +00:00
|
|
|
{
|
2020-05-20 09:40:49 +00:00
|
|
|
message_received += row_delimiter;
|
2020-06-07 11:14:05 +00:00
|
|
|
}
|
2020-06-05 13:42:11 +00:00
|
|
|
|
2020-06-04 06:22:53 +00:00
|
|
|
bool stop_loop = false;
|
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
/// Needed to avoid data race because this vector can be used at the same time by another thread in nextImpl().
|
2020-06-04 06:22:53 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
received.push_back(message_received);
|
|
|
|
|
|
|
|
/* As event loop is blocking to the thread that started it and a single thread should not be blocked while
|
2020-06-07 11:14:05 +00:00
|
|
|
* executing all callbacks on the connection (not only its own), then there should be some point to unblock.
|
|
|
|
* loop_started == 1 if current consumer is started the loop and not another.
|
2020-06-04 06:22:53 +00:00
|
|
|
*/
|
2020-06-07 11:14:05 +00:00
|
|
|
if (!loop_started)
|
2020-06-04 06:22:53 +00:00
|
|
|
{
|
|
|
|
stop_loop = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (stop_loop)
|
|
|
|
{
|
2020-06-07 11:14:05 +00:00
|
|
|
stopEventLoopWithTimeout();
|
2020-06-04 06:22:53 +00:00
|
|
|
}
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
|
|
|
{
|
2020-06-09 21:52:06 +00:00
|
|
|
consumer_error = true;
|
2020-06-04 06:22:53 +00:00
|
|
|
LOG_ERROR(log, "Consumer {} failed: {}", channel_id, message);
|
2020-05-20 09:40:49 +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-06-09 21:52:06 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::checkSubscription()
|
|
|
|
{
|
|
|
|
/// In general this condition will always be true and looping/resubscribing would not happen
|
|
|
|
if (count_subscribed == num_queues)
|
|
|
|
return;
|
2020-06-08 01:11:48 +00:00
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
wait_subscribed = num_queues;
|
2020-06-08 01:11:48 +00:00
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
/// These variables are updated in a separate thread
|
|
|
|
while (count_subscribed != wait_subscribed && !consumer_error)
|
|
|
|
{
|
|
|
|
startEventLoop(loop_started);
|
2020-06-08 01:11:48 +00:00
|
|
|
}
|
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
LOG_TRACE(log, "Consumer {} is subscribed to {} queues", channel_id, count_subscribed);
|
|
|
|
|
|
|
|
/// A case that would not normally happen
|
|
|
|
for (auto & queue : queues)
|
2020-06-08 01:11:48 +00:00
|
|
|
{
|
2020-06-09 21:52:06 +00:00
|
|
|
subscribe(queue);
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-06-04 06:22:53 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::stopEventLoop()
|
|
|
|
{
|
|
|
|
eventHandler.stop();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-06-07 11:14:05 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::stopEventLoopWithTimeout()
|
|
|
|
{
|
|
|
|
eventHandler.stopWithTimeout();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
void ReadBufferFromRabbitMQConsumer::startEventLoop(std::atomic<bool> & loop_started)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2020-06-09 21:52:06 +00:00
|
|
|
eventHandler.startConsumerLoop(loop_started);
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
bool ReadBufferFromRabbitMQConsumer::nextImpl()
|
|
|
|
{
|
|
|
|
if (stopped || !allowed)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
if (current == messages.end())
|
|
|
|
{
|
|
|
|
if (received.empty())
|
|
|
|
{
|
2020-06-07 11:14:05 +00:00
|
|
|
/// Run the onReceived callbacks to save the messages that have been received by now, blocks current thread
|
2020-06-09 21:52:06 +00:00
|
|
|
startEventLoop(loop_started);
|
2020-06-07 11:14:05 +00:00
|
|
|
loop_started = false;
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (received.empty())
|
|
|
|
{
|
2020-05-26 17:34:57 +00:00
|
|
|
LOG_TRACE(log, "No more messages to be fetched");
|
2020-05-20 09:40:49 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
messages.clear();
|
2020-05-29 16:04:44 +00:00
|
|
|
|
2020-06-09 21:52:06 +00:00
|
|
|
/// Needed to avoid data race because this vector can be used at the same time by another thread in onReceived callback.
|
2020-05-29 16:04:44 +00:00
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
|
2020-05-20 09:40:49 +00:00
|
|
|
messages.swap(received);
|
|
|
|
current = messages.begin();
|
|
|
|
}
|
|
|
|
|
2020-05-26 17:34:57 +00:00
|
|
|
auto * new_position = const_cast<char *>(current->data());
|
2020-05-20 09:40:49 +00:00
|
|
|
BufferBase::set(new_position, current->size(), 0);
|
|
|
|
|
|
|
|
++current;
|
|
|
|
allowed = false;
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|