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 <atomic>
|
|
|
|
#include <memory>
|
2022-10-28 16:41:10 +00:00
|
|
|
#include <Storages/RabbitMQ/RabbitMQConsumer.h>
|
2020-05-20 09:40:49 +00:00
|
|
|
#include <Storages/RabbitMQ/RabbitMQHandler.h>
|
2023-04-17 11:30:51 +00:00
|
|
|
#include <Storages/RabbitMQ/RabbitMQConnection.h>
|
2022-10-28 16:41:10 +00:00
|
|
|
#include <IO/ReadBufferFromMemory.h>
|
2022-04-27 15:05:45 +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
|
|
|
|
{
|
|
|
|
|
2021-10-08 08:48:08 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
RabbitMQConsumer::RabbitMQConsumer(
|
2021-09-10 10:28:09 +00:00
|
|
|
RabbitMQHandler & event_handler_,
|
2020-10-27 07:14:38 +00:00
|
|
|
std::vector<String> & queues_,
|
2020-08-15 06:50:53 +00:00
|
|
|
size_t channel_id_base_,
|
|
|
|
const String & channel_base_,
|
2020-05-20 09:40:49 +00:00
|
|
|
Poco::Logger * log_,
|
2023-04-03 10:11:12 +00:00
|
|
|
uint32_t queue_size_)
|
2022-10-28 16:41:10 +00:00
|
|
|
: event_handler(event_handler_)
|
2020-10-27 07:14:38 +00:00
|
|
|
, queues(queues_)
|
2020-08-15 06:50:53 +00:00
|
|
|
, channel_base(channel_base_)
|
|
|
|
, channel_id_base(channel_id_base_)
|
2020-07-13 01:11:35 +00:00
|
|
|
, log(log_)
|
2020-10-27 20:28:52 +00:00
|
|
|
, received(queue_size_)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
void RabbitMQConsumer::stop()
|
2022-06-23 18:43:35 +00:00
|
|
|
{
|
2023-04-03 10:18:04 +00:00
|
|
|
stopped = true;
|
2023-04-02 16:21:45 +00:00
|
|
|
cv.notify_one();
|
2023-04-06 13:43:16 +00:00
|
|
|
}
|
2023-04-03 10:11:12 +00:00
|
|
|
|
2023-04-06 13:43:16 +00:00
|
|
|
void RabbitMQConsumer::closeConnections()
|
|
|
|
{
|
2022-06-23 18:43:35 +00:00
|
|
|
if (consumer_channel)
|
|
|
|
consumer_channel->close();
|
|
|
|
}
|
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
void RabbitMQConsumer::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
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
LOG_TRACE(
|
|
|
|
log, "Consumer on channel {} ({}/{}) is subscribed to queue {}",
|
|
|
|
channel_id, subscriptions_num, queues.size(), queue_name);
|
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());
|
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
MessageData result{
|
|
|
|
.message = message_received,
|
|
|
|
.message_id = message.hasMessageID() ? message.messageID() : "",
|
|
|
|
.timestamp = message.hasTimestamp() ? message.timestamp() : 0,
|
|
|
|
.redelivered = redelivered,
|
|
|
|
.delivery_tag = delivery_tag,
|
|
|
|
.channel_id = channel_id};
|
|
|
|
|
|
|
|
if (!received.push(std::move(result)))
|
2021-10-07 22:06:33 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue");
|
2023-04-01 16:41:42 +00:00
|
|
|
|
|
|
|
cv.notify_one();
|
2020-07-29 19:45:18 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
.onError([&](const char * message)
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2023-04-17 11:30:51 +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 no one else is supposed to touch them
|
2020-08-28 08:52:02 +00:00
|
|
|
*/
|
2020-08-15 06:50:53 +00:00
|
|
|
LOG_ERROR(log, "Consumer failed on channel {}. Reason: {}", channel_id, message);
|
2023-04-17 11:30:51 +00:00
|
|
|
state = State::ERROR;
|
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
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
bool RabbitMQConsumer::ackMessages(const CommitInfo & commit_info)
|
2020-07-24 12:33:07 +00:00
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
if (state != State::OK)
|
|
|
|
return false;
|
2020-08-15 06:50:53 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
/// Nothing to ack.
|
|
|
|
if (!commit_info.delivery_tag)
|
|
|
|
return false;
|
2020-08-15 06:50:53 +00:00
|
|
|
|
2023-04-17 11:30:51 +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 (commit_info.channel_id != channel_id)
|
|
|
|
return false;
|
2020-08-28 08:52:02 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
/// Duplicate ack?
|
|
|
|
if (commit_info.delivery_tag > last_commited_delivery_tag
|
|
|
|
&& consumer_channel->ack(commit_info.delivery_tag, AMQP::multiple))
|
|
|
|
{
|
|
|
|
last_commited_delivery_tag = commit_info.delivery_tag;
|
2020-07-24 12:33:07 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
LOG_TRACE(
|
|
|
|
log, "Consumer committed messages with deliveryTags up to {} on channel {}",
|
|
|
|
last_commited_delivery_tag, channel_id);
|
2020-07-24 12:33:07 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
return true;
|
|
|
|
}
|
2020-08-15 06:50:53 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
LOG_ERROR(
|
|
|
|
log,
|
|
|
|
"Did not commit messages for {}:{}, (current commit point {}:{})",
|
|
|
|
commit_info.channel_id, commit_info.delivery_tag,
|
|
|
|
channel_id, last_commited_delivery_tag);
|
2020-08-15 06:50:53 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
return false;
|
2020-08-15 06:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
void RabbitMQConsumer::updateChannel(RabbitMQConnection & connection)
|
2020-08-15 06:50:53 +00:00
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
state = State::INITIALIZING;
|
|
|
|
last_commited_delivery_tag = 0;
|
2020-08-28 08:52:02 +00:00
|
|
|
|
2023-04-17 11:30:51 +00:00
|
|
|
consumer_channel = connection.createChannel();
|
2020-08-15 06:50:53 +00:00
|
|
|
consumer_channel->onReady([&]()
|
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
/// 1. channel_id_base - indicates current consumer buffer.
|
|
|
|
/// 2. channel_id_couner - indicates serial number of created channel for current buffer
|
|
|
|
/// (incremented on each channel update).
|
|
|
|
/// 3. channel_base is to guarantee that channel_id is unique for each table.
|
|
|
|
channel_id = fmt::format("{}_{}_{}", channel_id_base, channel_id_counter++, channel_base);
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Channel {} is successfully created", channel_id);
|
|
|
|
|
|
|
|
subscriptions_num = 0;
|
|
|
|
subscribe();
|
|
|
|
|
|
|
|
state = State::OK;
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
state = State::ERROR;
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
2020-08-15 06:50:53 +00:00
|
|
|
});
|
2020-08-28 08:52:02 +00:00
|
|
|
|
|
|
|
consumer_channel->onError([&](const char * message)
|
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
LOG_ERROR(log, "Channel {} in an error state: {}", channel_id, message);
|
|
|
|
state = State::ERROR;
|
2020-08-28 08:52:02 +00:00
|
|
|
});
|
2020-08-15 06:50:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
bool RabbitMQConsumer::needChannelUpdate()
|
2020-12-03 14:11:35 +00:00
|
|
|
{
|
2023-04-17 11:30:51 +00:00
|
|
|
chassert(consumer_channel);
|
|
|
|
return state == State::ERROR;
|
2020-12-03 14:11:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
ReadBufferPtr RabbitMQConsumer::consume()
|
2020-05-20 09:40:49 +00:00
|
|
|
{
|
2022-10-28 16:41:10 +00:00
|
|
|
if (stopped || !received.tryPop(current))
|
|
|
|
return nullptr;
|
2020-05-20 09:40:49 +00:00
|
|
|
|
2022-10-28 16:41:10 +00:00
|
|
|
return std::make_shared<ReadBufferFromMemory>(current.message.data(), current.message.size());
|
2020-05-20 09:40:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|