ClickHouse/src/Storages/RabbitMQ/StorageRabbitMQ.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

205 lines
7.2 KiB
C++
Raw Normal View History

2020-05-20 05:30:38 +00:00
#pragma once
2020-05-20 05:30:38 +00:00
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Poco/Semaphore.h>
#include <mutex>
#include <atomic>
#include <Storages/RabbitMQ/Buffer_fwd.h>
2020-08-31 09:12:36 +00:00
#include <Storages/RabbitMQ/RabbitMQSettings.h>
2021-09-10 10:28:09 +00:00
#include <Storages/RabbitMQ/RabbitMQConnection.h>
2020-08-15 06:50:53 +00:00
#include <Common/thread_local_rng.h>
#include <amqpcpp/libuv.h>
#include <uv.h>
2020-08-15 06:50:53 +00:00
#include <random>
2020-05-20 05:30:38 +00:00
2020-05-20 05:30:38 +00:00
namespace DB
{
class StorageRabbitMQ final: public IStorage, WithContext
2020-05-20 05:30:38 +00:00
{
public:
StorageRabbitMQ(
const StorageID & table_id_,
ContextPtr context_,
const ColumnsDescription & columns_,
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
bool is_attach_);
2020-05-20 05:30:38 +00:00
std::string getName() const override { return "RabbitMQ"; }
2020-06-01 15:37:23 +00:00
bool noPushingToViews() const override { return true; }
2020-05-20 05:30:38 +00:00
void startup() override;
void shutdown() override;
2021-05-04 16:26:47 +00:00
/// This is a bad way to let storage know in shutdown() that table is going to be dropped. There are some actions which need
/// to be done only when table is dropped (not when detached). Also connection must be closed only in shutdown, but those
/// actions require an open connection. Therefore there needs to be a way inside shutdown() method to know whether it is called
/// because of drop query. And drop() method is not suitable at all, because it will not only require to reopen connection, but also
/// it can be called considerable time after table is dropped (for example, in case of Atomic database), which is not appropriate for the case.
void checkTableCanBeDropped() const override { drop_table = true; }
2020-09-07 11:08:53 +00:00
/// Always return virtual columns in addition to required columns
2022-05-20 19:49:31 +00:00
void read(
QueryPlan & query_plan,
2020-07-02 16:44:04 +00:00
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
2020-07-02 16:44:04 +00:00
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
2021-07-23 14:25:35 +00:00
SinkToStoragePtr write(
2020-07-02 16:44:04 +00:00
const ASTPtr & query,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context) override;
2020-06-01 15:37:23 +00:00
void pushReadBuffer(ConsumerBufferPtr buf);
ConsumerBufferPtr popReadBuffer();
ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout);
2020-06-01 15:37:23 +00:00
ProducerBufferPtr createWriteBuffer();
const String & getFormatName() const { return format_name; }
NamesAndTypesList getVirtuals() const override;
String getExchange() const { return exchange_name; }
void unbindExchange();
2020-06-04 06:14:09 +00:00
2020-12-03 14:11:35 +00:00
bool updateChannel(ChannelPtr & channel);
2020-12-02 18:34:01 +00:00
void updateQueues(std::vector<String> & queues_) { queues_ = queues; }
2021-12-11 21:19:06 +00:00
void prepareChannelForBuffer(ConsumerBufferPtr buffer);
2021-11-06 19:02:01 +00:00
void incrementReader();
void decrementReader();
2020-05-20 05:30:38 +00:00
private:
2021-05-31 14:49:02 +00:00
ContextMutablePtr rabbitmq_context;
2020-08-31 09:12:36 +00:00
std::unique_ptr<RabbitMQSettings> rabbitmq_settings;
2020-05-20 05:30:38 +00:00
const String exchange_name;
const String format_name;
2020-08-31 09:12:36 +00:00
AMQP::ExchangeType exchange_type;
Names routing_keys;
2020-05-20 05:30:38 +00:00
char row_delimiter;
const String schema_name;
2020-05-20 05:30:38 +00:00
size_t num_consumers;
size_t num_queues;
String queue_base;
Names queue_settings_list;
/// For insert query. Mark messages as durable.
2020-07-25 11:14:46 +00:00
const bool persistent;
2020-05-20 05:30:38 +00:00
/// A table setting. It is possible not to perform any RabbitMQ setup, which is supposed to be consumer-side setup:
/// declaring exchanges, queues, bindings. Instead everything needed from RabbitMQ table is to connect to a specific queue.
/// This solution disables all optimizations and is not really optimal, but allows user to fully control all RabbitMQ setup.
bool use_user_setup;
2020-08-31 09:12:36 +00:00
bool hash_exchange;
2020-05-20 05:30:38 +00:00
Poco::Logger * log;
2021-09-10 10:28:09 +00:00
RabbitMQConnectionPtr connection; /// Connection for all consumers
RabbitMQConfiguration configuration;
2020-08-31 09:12:36 +00:00
size_t num_created_consumers = 0;
2020-05-20 05:30:38 +00:00
Poco::Semaphore semaphore;
2020-12-03 14:11:35 +00:00
std::mutex buffers_mutex;
std::vector<ConsumerBufferPtr> buffers; /// available buffers for RabbitMQ consumers
2020-05-20 05:30:38 +00:00
2020-08-31 16:34:16 +00:00
String unique_strbase; /// to make unique consumer channel id
2020-09-07 11:08:53 +00:00
/// maximum number of messages in RabbitMQ queue (x-max-length). Also used
/// to setup size of inner buffer for received messages
2020-09-01 14:11:34 +00:00
uint32_t queue_size;
String sharding_exchange, bridge_exchange, consumer_exchange;
2020-08-31 16:34:16 +00:00
size_t consumer_id = 0; /// counter for consumer buffer, needed for channel id
2021-11-02 19:47:27 +00:00
2020-10-27 07:14:38 +00:00
std::vector<String> queues;
2020-05-20 05:30:38 +00:00
2020-09-07 11:08:53 +00:00
std::once_flag flag; /// remove exchange only once
std::mutex task_mutex;
BackgroundSchedulePool::TaskHolder streaming_task;
BackgroundSchedulePool::TaskHolder looping_task;
2020-12-02 01:17:50 +00:00
BackgroundSchedulePool::TaskHolder connection_task;
uint64_t milliseconds_to_wait;
2021-11-06 19:02:01 +00:00
/**
* ( ͡° ͜ʖ ͡° )* Evil atomics:
*/
/// Needed for tell MV or producer background tasks
2021-11-02 19:47:27 +00:00
/// that they must finish as soon as possible.
std::atomic<bool> shutdown_called{false};
/// Counter for producer buffers, needed for channel id.
/// Needed to generate unique producer buffer identifiers.
std::atomic<size_t> producer_id = 1;
/// Has connection background task completed successfully?
/// It is started only once -- in constructor.
std::atomic<bool> rabbit_is_ready = false;
/// Allow to remove exchange only once.
std::atomic<bool> exchange_removed = false;
2021-11-06 19:02:01 +00:00
/// For select query we must be aware of the end of streaming
/// to be able to turn off the loop.
std::atomic<size_t> readers_count = 0;
2021-11-29 13:13:20 +00:00
std::atomic<bool> mv_attached = false;
2021-11-06 19:02:01 +00:00
/// In select query we start event loop, but do not stop it
/// after that select is finished. Then in a thread, which
/// checks for MV we also check if we have select readers.
/// If not - we turn off the loop. The checks are done under
/// mutex to avoid having a turned off loop when select was
/// started.
std::mutex loop_mutex;
2021-11-02 19:47:27 +00:00
2020-10-25 07:44:04 +00:00
size_t read_attempts = 0;
2021-05-04 16:26:47 +00:00
mutable bool drop_table = false;
2021-09-16 10:46:43 +00:00
bool is_attach;
ConsumerBufferPtr createReadBuffer();
2021-09-16 10:46:43 +00:00
void initializeBuffers();
bool initialized = false;
2020-09-07 11:08:53 +00:00
/// Functions working in the background
void streamingToViewsFunc();
void loopingFunc();
2020-12-02 01:17:50 +00:00
void connectionFunc();
2020-08-28 08:52:02 +00:00
2021-11-06 19:02:01 +00:00
void startLoop();
void stopLoop();
void stopLoopIfNoReaders();
2021-05-05 07:52:21 +00:00
static Names parseSettings(String settings_list);
2020-09-01 14:11:34 +00:00
static AMQP::ExchangeType defineExchangeType(String exchange_type_);
static String getTableBasedName(String name, const StorageID & table_id);
2021-06-02 16:52:33 +00:00
ContextMutablePtr addSettings(ContextPtr context) const;
2020-09-07 11:08:53 +00:00
size_t getMaxBlockSize() const;
2020-08-31 10:00:28 +00:00
void deactivateTask(BackgroundSchedulePool::TaskHolder & task, bool wait, bool stop_loop);
2020-08-31 09:12:36 +00:00
2020-12-02 01:17:50 +00:00
void initRabbitMQ();
2021-05-04 16:26:47 +00:00
void cleanupRabbitMQ() const;
2021-05-13 09:39:57 +00:00
void initExchange(AMQP::TcpChannel & rabbit_channel);
void bindExchange(AMQP::TcpChannel & rabbit_channel);
void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel);
bool streamToViews();
bool checkDependencies(const StorageID & table_id);
2020-08-15 06:50:53 +00:00
2021-06-02 16:52:33 +00:00
static String getRandomName()
2020-08-15 06:50:53 +00:00
{
std::uniform_int_distribution<int> distribution('a', 'z');
String random_str(32, ' ');
for (auto & c : random_str)
c = distribution(thread_local_rng);
return random_str;
}
2020-05-20 05:30:38 +00:00
};
}