StorageRabbitMQ.cpp 17.1 KB
Newer Older
1
#include <Storages/RabbitMQ/StorageRabbitMQ.h>
K
kssenii 已提交
2 3 4 5 6 7 8 9 10 11 12 13 14 15
#include <DataStreams/IBlockInputStream.h>
#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>
16 17
#include <Storages/RabbitMQ/RabbitMQSettings.h>
#include <Storages/RabbitMQ/RabbitMQBlockInputStream.h>
K
kssenii 已提交
18 19
#include <Storages/RabbitMQ/RabbitMQBlockOutputStream.h>
#include <Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h>
20
#include <Storages/RabbitMQ/RabbitMQHandler.h>
K
kssenii 已提交
21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36
#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>

K
kssenii 已提交
37 38
namespace DB
{
39

K
Fixes  
kssenii 已提交
40
enum
K
kssenii 已提交
41
{
42 43
    Connection_setup_sleep = 200,
    Connection_setup_retries_max = 1000
K
kssenii 已提交
44
};
K
kssenii 已提交
45 46 47 48 49 50 51

namespace ErrorCodes
{
    extern const int LOGICAL_ERROR;
    extern const int BAD_ARGUMENTS;
}

52

K
kssenii 已提交
53 54 55 56 57
StorageRabbitMQ::StorageRabbitMQ(
        const StorageID & table_id_,
        Context & context_,
        const ColumnsDescription & columns_,
        const String & host_port_,
K
kssenii 已提交
58
        const Names & routing_keys_,
K
kssenii 已提交
59 60 61
        const String & exchange_name_,
        const String & format_name_,
        char row_delimiter_,
K
kssenii 已提交
62
        const String & exchange_type_,
K
kssenii 已提交
63
        size_t num_consumers_,
K
kssenii 已提交
64 65
        size_t num_queues_,
        const bool use_transactional_channel_)
K
kssenii 已提交
66 67 68
        : IStorage(table_id_)
        , global_context(context_.getGlobalContext())
        , rabbitmq_context(Context(global_context))
K
kssenii 已提交
69
        , routing_keys(global_context.getMacros()->expand(routing_keys_))
K
kssenii 已提交
70 71 72 73 74
        , exchange_name(exchange_name_)
        , format_name(global_context.getMacros()->expand(format_name_))
        , row_delimiter(row_delimiter_)
        , num_consumers(num_consumers_)
        , num_queues(num_queues_)
K
kssenii 已提交
75
        , exchange_type(exchange_type_)
K
kssenii 已提交
76
        , use_transactional_channel(use_transactional_channel_)
K
kssenii 已提交
77
        , log(&Poco::Logger::get("StorageRabbitMQ (" + table_id_.table_name + ")"))
K
kssenii 已提交
78
        , semaphore(0, num_consumers_)
K
Fixes  
kssenii 已提交
79 80 81
        , login_password(std::make_pair(
                    rabbitmq_context.getConfigRef().getString("rabbitmq_username", "root"),
                    rabbitmq_context.getConfigRef().getString("rabbitmq_password", "clickhouse")))
K
kssenii 已提交
82
        , parsed_address(parseAddress(global_context.getMacros()->expand(host_port_), 5672))
83 84
        , evbase(event_base_new())
        , eventHandler(evbase, log)
K
Fixes  
kssenii 已提交
85 86
        , connection(&eventHandler, AMQP::Address(parsed_address.first, parsed_address.second,
                    AMQP::Login(login_password.first, login_password.second), "/"))
K
kssenii 已提交
87
{
88 89 90 91 92 93 94 95 96 97 98
    size_t cnt_retries = 0;
    while (!connection.ready() && ++cnt_retries != Connection_setup_retries_max)
    {
        event_base_loop(evbase, EVLOOP_NONBLOCK | EVLOOP_ONCE);
        std::this_thread::sleep_for(std::chrono::milliseconds(Connection_setup_sleep));
    }

    if (!connection.ready())
    {
        LOG_ERROR(log, "Cannot set up connection for consumer");
    }
99

100
    rabbitmq_context.makeQueryContext();
A
alesapin 已提交
101 102 103
    StorageInMemoryMetadata storage_metadata;
    storage_metadata.setColumns(columns_);
    setInMemoryMetadata(storage_metadata);
104

105 106 107
    task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); });
    task->deactivate();

K
Fixes  
kssenii 已提交
108
    bind_by_id = num_consumers > 1 || num_queues > 1;
109 110 111

    auto table_id = getStorageID();
    String table_name = table_id.table_name;
112 113

    /// Make sure that local exchange name is unique for each table and is not the same as client's exchange name
114
    local_exchange_name = exchange_name + "_" + table_name;
115 116 117 118 119
}


Pipes StorageRabbitMQ::read(
        const Names & column_names,
A
alesapin 已提交
120
        const StorageMetadataPtr & metadata_snapshot,
121 122 123 124 125 126 127 128 129 130 131 132 133 134
        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);

    for (size_t i = 0; i < num_created_consumers; ++i)
    {
A
alesapin 已提交
135 136 137
        pipes.emplace_back(
            std::make_shared<SourceFromInputStream>(std::make_shared<RabbitMQBlockInputStream>(
                    *this, metadata_snapshot, context, column_names, log)));
138 139
    }

K
kssenii 已提交
140
    LOG_DEBUG(log, "Starting reading {} streams", pipes.size());
141

142 143 144 145
    return pipes;
}


A
alesapin 已提交
146
BlockOutputStreamPtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context)
K
kssenii 已提交
147
{
A
alesapin 已提交
148
    return std::make_shared<RabbitMQBlockOutputStream>(*this, metadata_snapshot, context);
K
kssenii 已提交
149 150 151
}


152 153 154 155 156 157 158 159 160
void StorageRabbitMQ::startup()
{
    for (size_t i = 0; i < num_consumers; ++i)
    {
        try
        {
            pushReadBuffer(createReadBuffer());
            ++num_created_consumers;
        }
K
kssenii 已提交
161
        catch (const AMQP::Exception & e)
162
        {
K
kssenii 已提交
163 164
            std::cerr << e.what();
            throw;
165 166 167 168 169 170 171 172 173 174 175 176 177
        }
    }

    task->activateAndSchedule();
}


void StorageRabbitMQ::shutdown()
{
    stream_cancelled = true;

    for (size_t i = 0; i < num_created_consumers; ++i)
    {
K
kssenii 已提交
178
        popReadBuffer();
179 180
    }

K
kssenii 已提交
181
    connection.close();
182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225
    task->deactivate();
}


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;

K
kssenii 已提交
226 227
    ChannelPtr consumer_channel = std::make_shared<AMQP::TcpChannel>(&connection);

K
kssenii 已提交
228
    return std::make_shared<ReadBufferFromRabbitMQConsumer>(consumer_channel, eventHandler, exchange_name, routing_keys,
229
            next_channel_id, log, row_delimiter, bind_by_id, num_queues, exchange_type, local_exchange_name, stream_cancelled);
230 231 232
}


K
kssenii 已提交
233 234
ProducerBufferPtr StorageRabbitMQ::createWriteBuffer()
{
235
    return std::make_shared<WriteBufferToRabbitMQProducer>(parsed_address, login_password, routing_keys[0], local_exchange_name,
K
kssenii 已提交
236 237
            log, num_consumers * num_queues, bind_by_id, use_transactional_channel,
            row_delimiter ? std::optional<char>{row_delimiter} : std::nullopt, 1, 1024);
K
kssenii 已提交
238 239 240
}


241 242 243 244 245 246 247 248 249 250
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)
    {
251
        auto table = DatabaseCatalog::instance().tryGetTable(db_tab, global_context);
252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284
        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;

K
kssenii 已提交
285
                LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count);
286 287 288 289 290 291 292 293 294 295 296 297 298

                if (!streamToViews())
                    break;
            }
        }
    }
    catch (...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }

    /// Wait for attached views
    if (!stream_cancelled)
K
kssenii 已提交
299
        task->activateAndSchedule();
300 301 302 303 304 305
}


bool StorageRabbitMQ::streamToViews()
{
    auto table_id = getStorageID();
306
    auto table = DatabaseCatalog::instance().getTable(table_id, global_context);
307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322
    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);

    for (size_t i = 0; i < num_created_consumers; ++i)
    {
A
alesapin 已提交
323
        auto stream = std::make_shared<RabbitMQBlockInputStream>(*this, getInMemoryMetadataPtr(), rabbitmq_context, block_io.out->getHeader().getNames(), log);
324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349
        streams.emplace_back(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;
        stream->setLimits(limits);
    }

    // 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;
K
kssenii 已提交
350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380
}


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);
            }
        }

K
kssenii 已提交
381
        String routing_key_list = rabbitmq_settings.rabbitmq_routing_key_list.value;
K
kssenii 已提交
382 383
        if (args_count >= 2)
        {
K
kssenii 已提交
384 385 386 387 388 389 390 391 392
            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);
K
kssenii 已提交
393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451
        }

        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];
            }
        }

K
kssenii 已提交
452
        String exchange_type = rabbitmq_settings.rabbitmq_exchange_type.value;
K
kssenii 已提交
453 454
        if (args_count >= 6)
        {
K
kssenii 已提交
455 456
            engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[5], args.local_context);

K
kssenii 已提交
457
            const auto * ast = engine_args[5]->as<ASTLiteral>();
K
kssenii 已提交
458
            if (ast && ast->value.getType() == Field::Types::String)
K
kssenii 已提交
459
            {
K
kssenii 已提交
460
                exchange_type = safeGet<String>(ast->value);
K
kssenii 已提交
461 462
            }

K
kssenii 已提交
463 464 465
            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);
K
kssenii 已提交
466
        }
K
kssenii 已提交
467

K
kssenii 已提交
468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495
        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);
            }
        }

K
kssenii 已提交
496 497 498 499 500 501 502 503 504 505 506 507 508 509
        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);
            }
        }

K
kssenii 已提交
510
        return StorageRabbitMQ::create(
K
kssenii 已提交
511
                args.table_id, args.context, args.columns,
K
kssenii 已提交
512 513
                host_port, routing_keys, exchange, format, row_delimiter, exchange_type, num_consumers,
                num_queues, use_transactional_channel);
K
kssenii 已提交
514 515 516 517 518 519 520 521 522 523
    };

    factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });

}


NamesAndTypesList StorageRabbitMQ::getVirtuals() const
{
    return NamesAndTypesList{
K
kssenii 已提交
524
            {"_exchange", std::make_shared<DataTypeString>()}
K
kssenii 已提交
525 526 527 528
    };
}

}