ClickHouse/src/Storages/RabbitMQ/StorageRabbitMQ.cpp

792 lines
27 KiB
C++
Raw Normal View History

#include <Storages/RabbitMQ/StorageRabbitMQ.h>
2020-05-20 05:30:38 +00:00
#include <DataStreams/IBlockInputStream.h>
2020-07-02 14:35:10 +00:00
#include <DataStreams/ConvertingBlockInputStream.h>
2020-05-20 05:30:38 +00:00
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/RabbitMQ/RabbitMQSettings.h>
#include <Storages/RabbitMQ/RabbitMQBlockInputStream.h>
2020-06-01 15:37:23 +00:00
#include <Storages/RabbitMQ/RabbitMQBlockOutputStream.h>
#include <Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h>
#include <Storages/RabbitMQ/RabbitMQHandler.h>
2020-05-20 05:30:38 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/StorageMaterializedView.h>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/config_version.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <common/logger_useful.h>
#include <Common/quoteString.h>
#include <Common/parseAddress.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <amqpcpp.h>
2020-05-31 09:34:57 +00:00
namespace DB
{
static const auto CONNECT_SLEEP = 200;
static const auto RETRIES_MAX = 1000;
2020-07-02 16:44:04 +00:00
static const auto HEARTBEAT_RESCHEDULE_MS = 3000;
2020-05-20 05:30:38 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int CANNOT_CONNECT_RABBITMQ;
2020-05-20 05:30:38 +00:00
}
namespace ExchangeType
{
/// Note that default here means default by implementation and not by rabbitmq settings
static const String DEFAULT = "default";
static const String FANOUT = "fanout";
static const String DIRECT = "direct";
static const String TOPIC = "topic";
static const String HASH = "consistent_hash";
static const String HEADERS = "headers";
}
2020-05-20 05:30:38 +00:00
StorageRabbitMQ::StorageRabbitMQ(
const StorageID & table_id_,
Context & context_,
const ColumnsDescription & columns_,
const String & host_port_,
2020-06-11 09:23:23 +00:00
const Names & routing_keys_,
2020-05-20 05:30:38 +00:00
const String & exchange_name_,
const String & format_name_,
char row_delimiter_,
2020-06-10 23:01:47 +00:00
const String & exchange_type_,
2020-05-20 05:30:38 +00:00
size_t num_consumers_,
2020-06-11 21:03:53 +00:00
size_t num_queues_,
2020-07-23 11:45:01 +00:00
const bool use_transactional_channel_,
2020-07-24 12:33:07 +00:00
const String & queue_base_,
2020-07-25 11:14:46 +00:00
const String & deadletter_exchange_,
const bool persistent_)
2020-05-20 05:30:38 +00:00
: IStorage(table_id_)
, global_context(context_.getGlobalContext())
, rabbitmq_context(Context(global_context))
2020-06-11 09:23:23 +00:00
, routing_keys(global_context.getMacros()->expand(routing_keys_))
2020-05-20 05:30:38 +00:00
, exchange_name(exchange_name_)
, format_name(global_context.getMacros()->expand(format_name_))
, row_delimiter(row_delimiter_)
, num_consumers(num_consumers_)
, num_queues(num_queues_)
2020-06-11 21:03:53 +00:00
, use_transactional_channel(use_transactional_channel_)
2020-07-23 11:45:01 +00:00
, queue_base(queue_base_)
2020-07-24 12:33:07 +00:00
, deadletter_exchange(deadletter_exchange_)
2020-07-25 11:14:46 +00:00
, persistent(persistent_)
2020-06-01 20:48:24 +00:00
, log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")"))
2020-07-13 01:11:35 +00:00
, parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672))
2020-06-05 13:42:11 +00:00
, login_password(std::make_pair(
2020-07-03 08:02:35 +00:00
global_context.getConfigRef().getString("rabbitmq.username"),
global_context.getConfigRef().getString("rabbitmq.password")))
2020-07-13 01:11:35 +00:00
, semaphore(0, num_consumers_)
2020-05-20 05:30:38 +00:00
{
2020-06-25 09:44:39 +00:00
loop = std::make_unique<uv_loop_t>();
uv_loop_init(loop.get());
event_handler = std::make_shared<RabbitMQHandler>(loop.get(), log);
connection = std::make_shared<AMQP::TcpConnection>(event_handler.get(), AMQP::Address(parsed_address.first, parsed_address.second, AMQP::Login(login_password.first, login_password.second), "/"));
size_t cnt_retries = 0;
while (!connection->ready() && ++cnt_retries != RETRIES_MAX)
{
2020-07-02 16:44:04 +00:00
event_handler->iterateLoop();
std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP));
}
if (!connection->ready())
throw Exception("Cannot set up connection for consumers", ErrorCodes::CANNOT_CONNECT_RABBITMQ);
rabbitmq_context.makeQueryContext();
2020-06-24 17:32:57 +00:00
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
setInMemoryMetadata(storage_metadata);
streaming_task = global_context.getSchedulePool().createTask("RabbitMQStreamingTask", [this]{ threadFunc(); });
streaming_task->deactivate();
heartbeat_task = global_context.getSchedulePool().createTask("RabbitMQHeartbeatTask", [this]{ heartbeatFunc(); });
heartbeat_task->deactivate();
hash_exchange = num_consumers > 1 || num_queues > 1;
if (exchange_type_ != ExchangeType::DEFAULT)
{
if (exchange_type_ == ExchangeType::FANOUT) exchange_type = AMQP::ExchangeType::fanout;
else if (exchange_type_ == ExchangeType::DIRECT) exchange_type = AMQP::ExchangeType::direct;
else if (exchange_type_ == ExchangeType::TOPIC) exchange_type = AMQP::ExchangeType::topic;
else if (exchange_type_ == ExchangeType::HASH) exchange_type = AMQP::ExchangeType::consistent_hash;
else if (exchange_type_ == ExchangeType::HEADERS) exchange_type = AMQP::ExchangeType::headers;
else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS);
}
else
{
exchange_type = AMQP::ExchangeType::fanout;
}
if (exchange_type == AMQP::ExchangeType::headers)
{
std::vector<String> matching;
for (const auto & header : routing_keys)
{
boost::split(matching, header, [](char c){ return c == '='; });
bind_headers[matching[0]] = matching[1];
matching.clear();
}
}
auto table_id = getStorageID();
String table_name = table_id.table_name;
/// Make sure that local exchange name is unique for each table and is not the same as client's exchange name
local_exchange = exchange_name + "_" + table_name;
bridge_exchange = local_exchange + "_bridge";
/// One looping task for all consumers as they share the same connection == the same handler == the same event loop
looping_task = global_context.getSchedulePool().createTask("RabbitMQLoopingTask", [this]{ loopingFunc(); });
looping_task->deactivate();
}
void StorageRabbitMQ::heartbeatFunc()
{
if (!stream_cancelled)
{
2020-07-02 16:44:04 +00:00
LOG_TRACE(log, "Sending RabbitMQ heartbeat");
connection->heartbeat();
2020-07-02 16:44:04 +00:00
heartbeat_task->scheduleAfter(HEARTBEAT_RESCHEDULE_MS);
}
}
void StorageRabbitMQ::loopingFunc()
{
LOG_DEBUG(log, "Starting event looping iterations");
2020-07-02 16:44:04 +00:00
event_handler->startLoop();
}
void StorageRabbitMQ::initExchange()
{
/* Declare client's exchange of the specified type and bind it to hash-exchange (if it is not already hash-exchange), which
* will evenly distribute messages between all consumers.
*/
setup_channel->declareExchange(exchange_name, exchange_type, AMQP::durable)
.onError([&](const char * message)
{
throw Exception("Unable to declare exchange. Make sure specified exchange is not already declared. Error: "
+ std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
/// Bridge exchange is needed to easily disconnect consumer queues.
setup_channel->declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable + AMQP::autodelete)
.onError([&](const char * message)
{
throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
if (!hash_exchange)
{
consumer_exchange = bridge_exchange;
return;
}
/// Declare exchange for sharding.
AMQP::Table binding_arguments;
binding_arguments["hash-property"] = "message_id";
setup_channel->declareExchange(local_exchange, AMQP::consistent_hash, AMQP::durable + AMQP::autodelete, binding_arguments)
.onError([&](const char * message)
{
throw Exception("Unable to declare exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
setup_channel->bindExchange(bridge_exchange, local_exchange, routing_keys[0])
.onError([&](const char * message)
{
throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
consumer_exchange = local_exchange;
}
void StorageRabbitMQ::bindExchange()
{
std::atomic<bool> binding_created = false;
2020-07-24 12:33:07 +00:00
size_t bound_keys = 0;
/// Bridge exchange connects client's exchange with consumers' queues.
if (exchange_type == AMQP::ExchangeType::headers)
{
setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0], bind_headers)
.onSuccess([&]()
{
binding_created = true;
})
.onError([&](const char * message)
{
throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
}
else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash)
{
setup_channel->bindExchange(exchange_name, bridge_exchange, routing_keys[0])
.onSuccess([&]()
{
binding_created = true;
})
.onError([&](const char * message)
{
throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
}
else
{
for (const auto & routing_key : routing_keys)
{
setup_channel->bindExchange(exchange_name, bridge_exchange, routing_key)
.onSuccess([&]()
{
2020-07-24 12:33:07 +00:00
++bound_keys;
if (bound_keys == routing_keys.size())
binding_created = true;
})
.onError([&](const char * message)
{
throw Exception("Unable to bind exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
}
}
while (!binding_created)
{
event_handler->iterateLoop();
}
}
void StorageRabbitMQ::unbindExchange()
{
2020-07-25 11:14:46 +00:00
std::call_once(flag, [&]()
{
setup_channel->removeExchange(bridge_exchange)
.onSuccess([&]()
{
exchange_removed.store(true);
})
.onError([&](const char * message)
{
throw Exception("Unable to remove exchange. Reason: " + std::string(message), ErrorCodes::CANNOT_CONNECT_RABBITMQ);
});
2020-07-23 11:45:01 +00:00
while (!exchange_removed.load())
{
event_handler->iterateLoop();
}
event_handler->stop();
looping_task->deactivate();
2020-07-25 11:14:46 +00:00
});
}
Pipes StorageRabbitMQ::read(
const Names & column_names,
2020-06-24 17:32:57 +00:00
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & /* query_info */,
const Context & context,
QueryProcessingStage::Enum /* processed_stage */,
size_t /* max_block_size */,
unsigned /* num_streams */)
{
if (num_created_consumers == 0)
return {};
Pipes pipes;
pipes.reserve(num_created_consumers);
2020-07-02 14:35:10 +00:00
auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
for (size_t i = 0; i < num_created_consumers; ++i)
{
2020-07-02 14:35:10 +00:00
auto rabbit_stream = std::make_shared<RabbitMQBlockInputStream>(
2020-07-13 01:11:35 +00:00
*this, metadata_snapshot, context, column_names);
2020-07-02 14:35:10 +00:00
auto converting_stream = std::make_shared<ConvertingBlockInputStream>(
rabbit_stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Name);
pipes.emplace_back(std::make_shared<SourceFromInputStream>(converting_stream));
}
if (!loop_started)
{
loop_started = true;
looping_task->activateAndSchedule();
}
LOG_DEBUG(log, "Starting reading {} streams", pipes.size());
return pipes;
}
2020-06-24 17:32:57 +00:00
BlockOutputStreamPtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context)
2020-06-01 15:37:23 +00:00
{
2020-06-24 17:32:57 +00:00
return std::make_shared<RabbitMQBlockOutputStream>(*this, metadata_snapshot, context);
2020-06-01 15:37:23 +00:00
}
void StorageRabbitMQ::startup()
{
setup_channel = std::make_shared<AMQP::TcpChannel>(connection.get());
initExchange();
bindExchange();
for (size_t i = 0; i < num_consumers; ++i)
{
try
{
pushReadBuffer(createReadBuffer());
++num_created_consumers;
}
2020-05-31 09:34:57 +00:00
catch (const AMQP::Exception & e)
{
2020-05-31 09:34:57 +00:00
std::cerr << e.what();
throw;
}
}
streaming_task->activateAndSchedule();
heartbeat_task->activateAndSchedule();
}
void StorageRabbitMQ::shutdown()
{
stream_cancelled = true;
2020-07-02 14:35:10 +00:00
event_handler->stop();
2020-07-02 14:35:10 +00:00
looping_task->deactivate();
streaming_task->deactivate();
heartbeat_task->deactivate();
2020-07-02 14:35:10 +00:00
for (size_t i = 0; i < num_created_consumers; ++i)
{
popReadBuffer();
}
connection->close();
}
void StorageRabbitMQ::pushReadBuffer(ConsumerBufferPtr buffer)
{
std::lock_guard lock(mutex);
buffers.push_back(buffer);
semaphore.set();
}
ConsumerBufferPtr StorageRabbitMQ::popReadBuffer()
{
return popReadBuffer(std::chrono::milliseconds::zero());
}
ConsumerBufferPtr StorageRabbitMQ::popReadBuffer(std::chrono::milliseconds timeout)
{
// Wait for the first free buffer
if (timeout == std::chrono::milliseconds::zero())
semaphore.wait();
else
{
if (!semaphore.tryWait(timeout.count()))
return nullptr;
}
// Take the first available buffer from the list
std::lock_guard lock(mutex);
auto buffer = buffers.back();
buffers.pop_back();
return buffer;
}
ConsumerBufferPtr StorageRabbitMQ::createReadBuffer()
{
if (update_channel_id)
next_channel_id += num_queues;
update_channel_id = true;
ChannelPtr consumer_channel = std::make_shared<AMQP::TcpChannel>(connection.get());
2020-05-31 08:39:22 +00:00
return std::make_shared<ReadBufferFromRabbitMQConsumer>(
consumer_channel, setup_channel, event_handler, consumer_exchange, exchange_type, routing_keys,
2020-07-23 11:45:01 +00:00
next_channel_id, queue_base, log, row_delimiter, hash_exchange, num_queues,
2020-07-24 12:33:07 +00:00
local_exchange, deadletter_exchange, stream_cancelled);
}
2020-06-01 15:37:23 +00:00
ProducerBufferPtr StorageRabbitMQ::createWriteBuffer()
{
return std::make_shared<WriteBufferToRabbitMQProducer>(
parsed_address, global_context, login_password, routing_keys, exchange_name, exchange_type,
2020-07-25 11:14:46 +00:00
log, num_consumers * num_queues, use_transactional_channel, persistent,
row_delimiter ? std::optional<char>{row_delimiter} : std::nullopt, 1, 1024);
2020-06-01 15:37:23 +00:00
}
bool StorageRabbitMQ::checkDependencies(const StorageID & table_id)
{
// Check if all dependencies are attached
auto dependencies = DatabaseCatalog::instance().getDependencies(table_id);
if (dependencies.empty())
return true;
// Check the dependencies are ready?
for (const auto & db_tab : dependencies)
{
auto table = DatabaseCatalog::instance().tryGetTable(db_tab, global_context);
if (!table)
return false;
// If it materialized view, check it's target table
auto * materialized_view = dynamic_cast<StorageMaterializedView *>(table.get());
if (materialized_view && !materialized_view->tryGetTargetTable())
return false;
// Check all its dependencies
if (!checkDependencies(db_tab))
return false;
}
return true;
}
void StorageRabbitMQ::threadFunc()
{
try
{
auto table_id = getStorageID();
// Check if at least one direct dependency is attached
size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size();
if (dependencies_count)
{
// Keep streaming as long as there are attached views and streaming is not cancelled
while (!stream_cancelled && num_created_consumers > 0)
{
if (!checkDependencies(table_id))
break;
2020-05-26 20:43:20 +00:00
LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count);
if (!streamToViews())
break;
}
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// Wait for attached views
if (!stream_cancelled)
streaming_task->schedule();
}
bool StorageRabbitMQ::streamToViews()
{
auto table_id = getStorageID();
auto table = DatabaseCatalog::instance().getTable(table_id, global_context);
if (!table)
throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR);
// Create an INSERT query for streaming data
auto insert = std::make_shared<ASTInsertQuery>();
insert->table_id = table_id;
InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true);
auto block_io = interpreter.execute();
// Create a stream for each consumer and join them in a union stream
BlockInputStreams streams;
streams.reserve(num_created_consumers);
2020-07-02 14:35:10 +00:00
auto metadata_snapshot = getInMemoryMetadataPtr();
auto column_names = block_io.out->getHeader().getNames();
auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
for (size_t i = 0; i < num_created_consumers; ++i)
{
2020-07-13 01:11:35 +00:00
auto rabbit_stream = std::make_shared<RabbitMQBlockInputStream>(*this, metadata_snapshot, rabbitmq_context, column_names);
2020-07-02 14:35:10 +00:00
auto converting_stream = std::make_shared<ConvertingBlockInputStream>(rabbit_stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Name);
streams.emplace_back(converting_stream);
// Limit read batch to maximum block size to allow DDL
IBlockInputStream::LocalLimits limits;
const Settings & settings = global_context.getSettingsRef();
limits.speed_limits.max_execution_time = settings.stream_flush_interval_ms;
limits.timeout_overflow_mode = OverflowMode::BREAK;
2020-07-02 14:35:10 +00:00
rabbit_stream->setLimits(limits);
}
if (!loop_started)
{
loop_started = true;
looping_task->activateAndSchedule();
}
// Join multiple streams if necessary
BlockInputStreamPtr in;
if (streams.size() > 1)
in = std::make_shared<UnionBlockInputStream>(streams, nullptr, streams.size());
else
in = streams[0];
std::atomic<bool> stub = {false};
copyData(*in, *block_io.out, &stub);
// Check whether the limits were applied during query execution
bool limits_applied = false;
const BlockStreamProfileInfo & info = in->getProfileInfo();
limits_applied = info.hasAppliedLimit();
return limits_applied;
2020-05-20 05:30:38 +00:00
}
void registerStorageRabbitMQ(StorageFactory & factory)
{
auto creator_fn = [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
size_t args_count = engine_args.size();
bool has_settings = args.storage_def->settings;
RabbitMQSettings rabbitmq_settings;
if (has_settings)
{
rabbitmq_settings.loadFromQuery(*args.storage_def);
}
String host_port = rabbitmq_settings.rabbitmq_host_port;
if (args_count >= 1)
{
const auto * ast = engine_args[0]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
host_port = safeGet<String>(ast->value);
}
else
{
throw Exception(String("RabbitMQ host:port must be a string"), ErrorCodes::BAD_ARGUMENTS);
}
}
2020-06-11 09:23:23 +00:00
String routing_key_list = rabbitmq_settings.rabbitmq_routing_key_list.value;
2020-05-20 05:30:38 +00:00
if (args_count >= 2)
{
2020-06-11 09:23:23 +00:00
engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.local_context);
routing_key_list = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
}
Names routing_keys;
boost::split(routing_keys, routing_key_list, [](char c){ return c == ','; });
for (String & key : routing_keys)
{
boost::trim(key);
2020-05-20 05:30:38 +00:00
}
String exchange = rabbitmq_settings.rabbitmq_exchange_name.value;
if (args_count >= 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
const auto * ast = engine_args[2]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
exchange = safeGet<String>(ast->value);
}
}
String format = rabbitmq_settings.rabbitmq_format.value;
if (args_count >= 4)
{
engine_args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[3], args.local_context);
const auto * ast = engine_args[3]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
format = safeGet<String>(ast->value);
}
else
{
throw Exception("Format must be a string", ErrorCodes::BAD_ARGUMENTS);
}
}
char row_delimiter = rabbitmq_settings.rabbitmq_row_delimiter;
if (args_count >= 5)
{
engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context);
const auto * ast = engine_args[4]->as<ASTLiteral>();
String arg;
if (ast && ast->value.getType() == Field::Types::String)
{
arg = safeGet<String>(ast->value);
}
else
{
throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS);
}
if (arg.size() > 1)
{
throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS);
}
else if (arg.empty())
{
row_delimiter = '\0';
}
else
{
row_delimiter = arg[0];
}
}
2020-06-10 23:01:47 +00:00
String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value;
2020-05-20 05:30:38 +00:00
if (args_count >= 6)
{
2020-06-10 23:01:47 +00:00
engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context);
2020-05-20 05:30:38 +00:00
const auto * ast = engine_args[5]->as<ASTLiteral>();
2020-06-10 23:01:47 +00:00
if (ast && ast->value.getType() == Field::Types::String)
2020-05-20 05:30:38 +00:00
{
2020-06-10 23:01:47 +00:00
exchange_type = safeGet<String>(ast->value);
2020-05-20 05:30:38 +00:00
}
2020-06-13 21:37:37 +00:00
if (exchange_type != "fanout" && exchange_type != "direct" && exchange_type != "topic"
&& exchange_type != "headers" && exchange_type != "consistent_hash")
throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS);
2020-06-11 20:05:35 +00:00
}
2020-06-10 23:01:47 +00:00
2020-05-20 05:30:38 +00:00
UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers;
if (args_count >= 7)
{
const auto * ast = engine_args[6]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
num_consumers = safeGet<UInt64>(ast->value);
}
else
{
throw Exception("Number of consumers must be a positive integer", ErrorCodes::BAD_ARGUMENTS);
}
}
UInt64 num_queues = rabbitmq_settings.rabbitmq_num_queues;
if (args_count >= 8)
{
const auto * ast = engine_args[7]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
num_consumers = safeGet<UInt64>(ast->value);
}
else
{
throw Exception("Number of queues must be a positive integer", ErrorCodes::BAD_ARGUMENTS);
}
}
2020-06-11 21:03:53 +00:00
bool use_transactional_channel = static_cast<bool>(rabbitmq_settings.rabbitmq_transactional_channel);
if (args_count >= 9)
{
const auto * ast = engine_args[8]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
use_transactional_channel = static_cast<bool>(safeGet<UInt64>(ast->value));
}
else
{
throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS);
}
}
2020-07-23 11:45:01 +00:00
String queue_base = rabbitmq_settings.rabbitmq_queue_base.value;
if (args_count >= 10)
{
engine_args[9] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[9], args.local_context);
const auto * ast = engine_args[9]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
{
queue_base = safeGet<String>(ast->value);
}
}
2020-07-24 12:33:07 +00:00
String deadletter_exchange = rabbitmq_settings.rabbitmq_deadletter_exchange.value;
if (args_count >= 11)
{
engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context);
2020-07-25 11:14:46 +00:00
const auto * ast = engine_args[10]->as<ASTLiteral>();
2020-07-24 12:33:07 +00:00
if (ast && ast->value.getType() == Field::Types::String)
{
deadletter_exchange = safeGet<String>(ast->value);
}
}
2020-07-25 11:14:46 +00:00
bool persistent = static_cast<bool>(rabbitmq_settings.rabbitmq_persistent_mode);
if (args_count >= 12)
{
const auto * ast = engine_args[11]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
persistent = static_cast<bool>(safeGet<UInt64>(ast->value));
}
else
{
throw Exception("Transactional channel parameter is a bool", ErrorCodes::BAD_ARGUMENTS);
}
}
2020-05-31 08:39:22 +00:00
return StorageRabbitMQ::create(
2020-05-31 09:34:57 +00:00
args.table_id, args.context, args.columns,
2020-06-11 21:03:53 +00:00
host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers,
2020-07-25 11:14:46 +00:00
num_queues, use_transactional_channel, queue_base, deadletter_exchange, persistent);
2020-05-20 05:30:38 +00:00
};
factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });
}
NamesAndTypesList StorageRabbitMQ::getVirtuals() const
{
return NamesAndTypesList{
2020-07-20 10:05:00 +00:00
{"_exchange_name", std::make_shared<DataTypeString>()},
{"_consumer_tag", std::make_shared<DataTypeString>()},
{"_delivery_tag", std::make_shared<DataTypeUInt64>()},
{"_redelivered", std::make_shared<DataTypeUInt8>()}
2020-05-20 05:30:38 +00:00
};
}
}