From a0c8d34775d3d1d9862270f707cf6df4a5ea45e5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 19 Dec 2022 21:07:44 +0100 Subject: [PATCH 01/78] RabbitMQ: respect flushing data by flush interval and by block size --- src/Storages/RabbitMQ/RabbitMQSettings.h | 2 +- src/Storages/RabbitMQ/RabbitMQSource.cpp | 41 ++-- src/Storages/RabbitMQ/RabbitMQSource.h | 11 +- .../ReadBufferFromRabbitMQConsumer.cpp | 2 +- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.h | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 19 +- .../integration/test_storage_rabbitmq/test.py | 221 +++++++++++++++++- 7 files changed, 271 insertions(+), 33 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 38baeadad2a..e54445cf64e 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -24,7 +24,7 @@ namespace DB M(String, rabbitmq_address, "", "Address for connection", 0) \ M(UInt64, rabbitmq_skip_broken_messages, 0, "Skip at least this number of broken messages from RabbitMQ per block", 0) \ M(UInt64, rabbitmq_max_block_size, 0, "Number of row collected before flushing data from RabbitMQ.", 0) \ - M(Milliseconds, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \ + M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \ M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \ M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \ M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \ diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index f6e5bb84037..b997e74abfc 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -61,6 +62,7 @@ RabbitMQSource::RabbitMQSource( , ack_in_suffix(ack_in_suffix_) , non_virtual_header(std::move(headers.first)) , virtual_header(std::move(headers.second)) + , log(&Poco::Logger::get("RabbitMQSource")) { storage.incrementReader(); } @@ -106,17 +108,15 @@ Chunk RabbitMQSource::generate() return chunk; } -bool RabbitMQSource::checkTimeLimit() const +bool RabbitMQSource::isTimeLimitExceeded() const { - if (max_execution_time != 0) + if (max_execution_time_ms != 0) { - auto elapsed_ns = total_stopwatch.elapsed(); - - if (elapsed_ns > static_cast(max_execution_time.totalMicroseconds()) * 1000) - return false; + uint64_t elapsed_time_ms = total_stopwatch.elapsedMilliseconds(); + return max_execution_time_ms <= elapsed_time_ms; } - return true; + return false; } Chunk RabbitMQSource::generateImpl() @@ -127,9 +127,11 @@ Chunk RabbitMQSource::generateImpl() buffer = storage.popReadBuffer(timeout); } - if (!buffer || is_finished) + if (is_finished || !buffer || buffer->isConsumerStopped()) return {}; + /// Currently it is one time usage source: to make sure data is flushed + /// strictly by timeout or by block size. is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); @@ -137,15 +139,17 @@ Chunk RabbitMQSource::generateImpl() storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); - size_t total_rows = 0; + buffer->allowNext(); while (true) { - if (buffer->queueEmpty()) - break; + size_t new_rows = 0; - auto new_rows = executor.execute(); + if (!buffer->hasPendingMessages()) + { + new_rows = executor.execute(); + } if (new_rows) { @@ -168,15 +172,20 @@ Chunk RabbitMQSource::generateImpl() virtual_columns[5]->insert(timestamp); } - total_rows = total_rows + new_rows; + total_rows += new_rows; } - buffer->allowNext(); - - if (total_rows >= max_block_size || buffer->queueEmpty() || buffer->isConsumerStopped() || !checkTimeLimit()) + if (total_rows >= max_block_size || buffer->isConsumerStopped() || isTimeLimitExceeded()) break; + + buffer->allowNext(); } + LOG_TEST( + log, + "Flushing {} rows (max block size: {}, time: {} / {} ms)", + total_rows, max_block_size, total_stopwatch.elapsedMilliseconds(), max_execution_time_ms); + if (total_rows == 0) return {}; diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index bd2882d1938..721a98aa2b2 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -27,13 +27,12 @@ public: Chunk generate() override; - bool queueEmpty() const { return !buffer || buffer->queueEmpty(); } + bool queueEmpty() const { return !buffer || buffer->hasPendingMessages(); } bool needChannelUpdate(); void updateChannel(); bool sendAck(); - - void setTimeLimit(Poco::Timespan max_execution_time_) { max_execution_time = max_execution_time_; } + void setTimeLimit(uint64_t max_execution_time_ms_) { max_execution_time_ms = max_execution_time_ms_; } private: StorageRabbitMQ & storage; @@ -47,12 +46,14 @@ private: const Block non_virtual_header; const Block virtual_header; + Poco::Logger * log; + ConsumerBufferPtr buffer; - Poco::Timespan max_execution_time = 0; + uint64_t max_execution_time_ms = 0; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; - bool checkTimeLimit() const; + bool isTimeLimitExceeded() const; RabbitMQSource( StorageRabbitMQ & storage_, diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 3543085f5a0..ba69b66ee4a 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -183,8 +183,8 @@ bool ReadBufferFromRabbitMQConsumer::nextImpl() { auto * new_position = const_cast(current.message.data()); BufferBase::set(new_position, current.message.size(), 0); - allowed = false; + allowed = false; return true; } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h index bd55d169744..f157cef7a4d 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h @@ -67,8 +67,9 @@ public: bool ackMessages(); void updateAckTracker(AckTracker record = AckTracker()); - bool queueEmpty() { return received.empty(); } - void allowNext() { allowed = true; } // Allow to read next message. + bool hasPendingMessages() { return received.empty(); } + + void allowNext() { allowed = true; } auto getChannelID() const { return current.track.channel_id; } auto getDeliveryTag() const { return current.track.delivery_tag; } @@ -89,7 +90,6 @@ private: const size_t channel_id_base; Poco::Logger * log; char row_delimiter; - bool allowed = true; const std::atomic & stopped; String channel_id; @@ -100,6 +100,8 @@ private: AckTracker last_inserted_record_info; UInt64 prev_tag = 0, channel_id_counter = 0; + + bool allowed = false; }; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index bce3fee71f7..07fa8bac189 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -715,6 +715,12 @@ void StorageRabbitMQ::read( auto rabbit_source = std::make_shared( *this, storage_snapshot, modified_context, column_names, 1, rabbitmq_settings->rabbitmq_commit_on_select); + uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed + ? rabbitmq_settings->rabbitmq_flush_interval_ms + : (static_cast(getContext()->getSettingsRef().stream_flush_interval_ms) * 1000); + + rabbit_source->setTimeLimit(max_execution_time_ms); + auto converting_dag = ActionsDAG::makeConvertingActions( rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), sample_block.getColumnsWithTypeAndName(), @@ -1075,14 +1081,15 @@ bool StorageRabbitMQ::streamToViews() { auto source = std::make_shared( *this, storage_snapshot, rabbitmq_context, column_names, block_size, false); + + uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed + ? rabbitmq_settings->rabbitmq_flush_interval_ms + : (static_cast(getContext()->getSettingsRef().stream_flush_interval_ms) * 1000); + + source->setTimeLimit(max_execution_time_ms); + sources.emplace_back(source); pipes.emplace_back(source); - - Poco::Timespan max_execution_time = rabbitmq_settings->rabbitmq_flush_interval_ms.changed - ? rabbitmq_settings->rabbitmq_flush_interval_ms - : getContext()->getSettingsRef().stream_flush_interval_ms; - - source->setTimeLimit(max_execution_time); } block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index f5d216a8b92..ddb7a185c2f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -153,6 +153,7 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): rabbitmq_exchange_name = 'empty', rabbitmq_commit_on_select = 1, rabbitmq_format = 'TSV', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; """.format( rabbitmq_cluster.rabbitmq_host @@ -169,6 +170,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_commit_on_select = 1, + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' """.format( @@ -221,6 +223,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): rabbitmq_exchange_name = 'csv', rabbitmq_commit_on_select = 1, rabbitmq_format = 'CSV', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; """ ) @@ -262,6 +265,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_exchange_name = 'tsv', rabbitmq_format = 'TSV', rabbitmq_commit_on_select = 1, + rabbitmq_flush_interval_ms=1000, rabbitmq_queue_base = 'tsv', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -303,6 +307,7 @@ def test_rabbitmq_macros(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{rabbitmq_host}:{rabbitmq_port}', rabbitmq_commit_on_select = 1, + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = '{rabbitmq_exchange_name}', rabbitmq_format = '{rabbitmq_format}' """ @@ -342,6 +347,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mv', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() @@ -398,6 +404,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mvsq', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -441,6 +448,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mmv', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view1 (key UInt64, value UInt64) @@ -498,6 +506,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'pb', rabbitmq_format = 'Protobuf', + rabbitmq_flush_interval_ms=1000, rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() @@ -572,6 +581,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'big', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) ENGINE = MergeTree @@ -608,6 +618,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_exchange_name = 'test_sharding', rabbitmq_num_queues = 10, rabbitmq_num_consumers = 10, + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) @@ -685,6 +696,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'combo', rabbitmq_queue_base = 'combo', + rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = 2, rabbitmq_num_queues = 5, rabbitmq_format = 'JSONEachRow', @@ -778,6 +790,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'insert', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert1', rabbitmq_format = 'TSV', @@ -835,6 +848,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'insert_headers', rabbitmq_exchange_type = 'headers', + rabbitmq_flush_interval_ms=1000, rabbitmq_routing_key_list = 'test=insert,topic=headers', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -901,6 +915,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_exchange_name = 'many_inserts', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) @@ -909,6 +924,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_exchange_name = 'many_inserts', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; """ @@ -988,6 +1004,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_queue_base = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_num_consumers = 5, + rabbitmq_flush_interval_ms=1000, rabbitmq_num_queues = 10, rabbitmq_max_block_size = 10000, rabbitmq_routing_key_list = 'over', @@ -999,6 +1016,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_overload (key UInt64, value UInt64) @@ -1084,6 +1102,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'direct_exchange_testing', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'direct_{0}', @@ -1175,6 +1194,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_routing_key_list = 'key_{0}', rabbitmq_exchange_name = 'fanout_exchange_testing', rabbitmq_exchange_type = 'fanout', @@ -1261,6 +1281,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.{0}', @@ -1284,6 +1305,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.logs', @@ -1385,6 +1407,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): rabbitmq_exchange_type = 'consistent_hash', rabbitmq_exchange_name = 'hash_exchange_testing', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS SELECT key, value, _channel_id AS channel_id FROM test.{0}; @@ -1482,6 +1505,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): rabbitmq_exchange_name = 'multiple_bindings_testing', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.bindings_mv TO test.destination AS @@ -1571,6 +1595,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_num_consumers = 2, rabbitmq_exchange_name = 'headers_exchange_testing', rabbitmq_exchange_type = 'headers', + rabbitmq_flush_interval_ms=1000, rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2020', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1595,6 +1620,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_exchange_type = 'headers', rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2019', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS SELECT key, value FROM test.headers_exchange_{0}; @@ -1667,6 +1693,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT value, key, _exchange_name, _channel_id, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; @@ -1735,6 +1762,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals_mv', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value UInt64, exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() @@ -1820,6 +1848,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): rabbitmq_exchange_name = 'many_consumers', rabbitmq_num_queues = 2, rabbitmq_num_consumers = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_queue_base = 'many_consumers', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1909,6 +1938,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): CREATE TABLE test.consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', rabbitmq_num_consumers = 2, @@ -1921,6 +1951,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_persistent = '1', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; """ @@ -1984,6 +2015,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'consumer_reconnect', rabbitmq_num_consumers = 10, + rabbitmq_flush_interval_ms=1000, rabbitmq_num_queues = 10, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -2064,6 +2096,7 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): rabbitmq_exchange_name = 'block', rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'block', + rabbitmq_flush_interval_ms=1000, rabbitmq_max_block_size = 100, rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -2144,6 +2177,7 @@ def test_rabbitmq_no_connection_at_startup_1(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'cs', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = '5', rabbitmq_row_delimiter = '\\n'; """ @@ -2160,6 +2194,7 @@ def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster): rabbitmq_exchange_name = 'cs', rabbitmq_format = 'JSONEachRow', rabbitmq_num_consumers = '5', + rabbitmq_flush_interval_ms=1000, rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree @@ -2216,6 +2251,7 @@ def test_rabbitmq_format_factory_settings(rabbitmq_cluster): ) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'format_settings', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', date_time_input_format = 'best_effort'; """ @@ -2278,6 +2314,7 @@ def test_rabbitmq_vhost(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'vhost', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_vhost = '/' """ ) @@ -2306,6 +2343,7 @@ def test_rabbitmq_drop_table_properly(rabbitmq_cluster): CREATE TABLE test.rabbitmq_drop (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'drop', rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'rabbit_queue_drop' @@ -2352,6 +2390,7 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'rabbit_exchange', + rabbitmq_flush_interval_ms=1000, rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'rabbit_queue_settings', rabbitmq_queue_settings_list = 'x-max-length=10,x-overflow=reject-publish' @@ -2433,6 +2472,7 @@ def test_rabbitmq_queue_consume(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'rabbit_queue', + rabbitmq_flush_interval_ms=1000, rabbitmq_queue_consume = 1; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; @@ -2467,6 +2507,7 @@ def test_rabbitmq_produce_consume_avro(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_format = 'Avro', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'avro', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'avro'; @@ -2475,6 +2516,7 @@ def test_rabbitmq_produce_consume_avro(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_format = 'Avro', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'avro', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'avro'; @@ -2517,6 +2559,7 @@ def test_rabbitmq_bad_args(rabbitmq_cluster): CREATE TABLE test.drop (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = 'f', rabbitmq_format = 'JSONEachRow'; """ @@ -2529,6 +2572,7 @@ def test_rabbitmq_issue_30691(rabbitmq_cluster): CREATE TABLE test.rabbitmq_drop (json String) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_flush_interval_ms=1000, rabbitmq_exchange_name = '30691', rabbitmq_row_delimiter = '\\n', -- Works only if adding this setting rabbitmq_format = 'LineAsString', @@ -2589,6 +2633,7 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mv', rabbitmq_format = 'JSONEachRow', + rabbitmq_flush_interval_ms=1000, rabbitmq_queue_base = 'drop_mv'; """ ) @@ -2679,6 +2724,7 @@ def test_rabbitmq_random_detach(rabbitmq_cluster): rabbitmq_exchange_name = 'random', rabbitmq_queue_base = 'random', rabbitmq_num_queues = 2, + rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = 2, rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) @@ -2743,7 +2789,9 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): instance.query( """ CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ(rabbit1, rabbitmq_vhost = '/') """ + ENGINE = RabbitMQ(rabbit1, rabbitmq_vhost = '/') + SETTINGS rabbitmq_flush_interval_ms=1000; + """ ) channel.basic_publish( @@ -2779,6 +2827,7 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): settings rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'xhep', rabbitmq_format = 'MsgPack', + rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = 1; create table rabbit_out (val String) @@ -2786,6 +2835,7 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): settings rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'xhep', rabbitmq_format = 'MsgPack', + rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = 1; set stream_like_engine_allow_direct_select=1; insert into rabbit_out select 'kek'; @@ -2821,12 +2871,14 @@ def test_rabbitmq_address(rabbitmq_cluster): SETTINGS rabbitmq_exchange_name = 'rxhep', rabbitmq_format = 'CSV', rabbitmq_num_consumers = 1, + rabbitmq_flush_interval_ms=1000, rabbitmq_address='amqp://root:clickhouse@rabbitmq1:5672/'; create table rabbit_out (val String) engine=RabbitMQ SETTINGS rabbitmq_exchange_name = 'rxhep', rabbitmq_format = 'CSV', rabbitmq_num_consumers = 1, + rabbitmq_flush_interval_ms=1000, rabbitmq_address='amqp://root:clickhouse@rabbitmq1:5672/'; set stream_like_engine_allow_direct_select=1; insert into rabbit_out select 'kek'; @@ -2848,3 +2900,170 @@ def test_rabbitmq_address(rabbitmq_cluster): instance2.query("drop table rabbit_in sync") instance2.query("drop table rabbit_out sync") + + +def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'flush_by_block', + rabbitmq_queue_base = 'flush_by_block', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 640000, /* should not flush by time during test */ + rabbitmq_format = 'JSONEachRow'; + + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq; + """ + ) + + cancel = threading.Event() + + def produce(): + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + connection = pika.BlockingConnection(parameters) + + while not cancel.is_set(): + try: + channel = connection.channel() + channel.basic_publish( + exchange="flush_by_block", + routing_key="", + body=json.dumps({"key": 0, "value": 0}), + ) + except e: + print(f"Got error: {str(e)}") + + produce_thread = threading.Thread(target=produce) + produce_thread.start() + + while 0 == int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ): + time.sleep(0.5) + + cancel.set() + produce_thread.join() + + # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). + result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") + # logging.debug(result) + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.rabbitmq; + """ + ) + + # 100 = first poll should return 100 messages (and rows) + # not waiting for stream_flush_interval_ms + assert ( + int(result) == 100 + ), "Messages from rabbitmq should be flushed when block of size rabbitmq_max_block_size is formed!" + + +def test_rabbitmq_flush_by_time(rabbitmq_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'flush_by_time', + rabbitmq_queue_base = 'flush_by_time', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 5000, + rabbitmq_format = 'JSONEachRow'; + + CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) + ENGINE = MergeTree() + ORDER BY key; + """ + ) + + cancel = threading.Event() + + def produce(): + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + connection = pika.BlockingConnection(parameters) + + while not cancel.is_set(): + try: + channel = connection.channel() + channel.basic_publish( + exchange="flush_by_time", + routing_key="", + body=json.dumps({"key": 0, "value": 0}), + ) + print("Produced a message") + time.sleep(0.8) + except e: + print(f"Got error: {str(e)}") + + produce_thread = threading.Thread(target=produce) + produce_thread.start() + + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq; + """ + ) + + while True: + time.sleep(0.2) + count = instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view'" + ) + print(f"kssenii total count: {count}") + count = int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ) + print(f"kssenii count: {count}") + if count > 0: + break + + time.sleep(12) + result = instance.query("SELECT uniqExact(ts) FROM test.view") + + cancel.set() + produce_thread.join() + + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.rabbitmq; + """ + ) + + assert int(result) == 3 From 499565a604fe7cab5a05eb826c1f6441ee6e997d Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Dec 2022 19:32:59 +0100 Subject: [PATCH 02/78] Fix tests --- .../integration/test_storage_rabbitmq/test.py | 47 +++++++++++++------ 1 file changed, 33 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index ddb7a185c2f..cc86f63c2e3 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -49,6 +49,7 @@ def rabbitmq_check_result(result, check=False, ref_file="test_rabbitmq_json.refe def wait_rabbitmq_to_start(rabbitmq_docker_id, timeout=180): + logging.getLogger("pika").propagate = False start = time.time() while time.time() - start < timeout: try: @@ -609,6 +610,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): NUM_CONSUMERS = 10 NUM_QUEUES = 10 + logging.getLogger("pika").propagate = False instance.query( """ @@ -618,7 +620,8 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_exchange_name = 'test_sharding', rabbitmq_num_queues = 10, rabbitmq_num_consumers = 10, - rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms=10000, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) @@ -671,8 +674,10 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): while True: result1 = instance.query("SELECT count() FROM test.view") time.sleep(1) - if int(result1) == messages_num * threads_num: + expected = messages_num * threads_num + if int(result1) == expected: break + print(f"Result {result1} / {expected}") result2 = instance.query("SELECT count(DISTINCT channel_id) FROM test.view") @@ -688,6 +693,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): def test_rabbitmq_mv_combo(rabbitmq_cluster): NUM_MV = 5 NUM_CONSUMERS = 4 + logging.getLogger("pika").propagate = False instance.query( """ @@ -696,6 +702,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'combo', rabbitmq_queue_base = 'combo', + rabbitmq_max_block_size = 100, rabbitmq_flush_interval_ms=1000, rabbitmq_num_consumers = 2, rabbitmq_num_queues = 5, @@ -761,8 +768,10 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): result += int( instance.query("SELECT count() FROM test.combo_{0}".format(mv_id)) ) - if int(result) == messages_num * threads_num * NUM_MV: + expected = messages_num * threads_num * NUM_MV + if int(result) == expected: break + print(f"Result: {result} / {expected}") time.sleep(1) for thread in threads: @@ -1005,8 +1014,8 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_num_consumers = 5, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size = 100, rabbitmq_num_queues = 10, - rabbitmq_max_block_size = 10000, rabbitmq_routing_key_list = 'over', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -1017,6 +1026,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size = 100, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_overload (key UInt64, value UInt64) @@ -1059,8 +1069,10 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): while True: result = instance.query("SELECT count() FROM test.view_overload") time.sleep(1) - if int(result) == messages_num * threads_num: + expected = messages_num * threads_num + if int(result) == expected: break + print(f"Result: {result} / {expected}") instance.query( """ @@ -1849,6 +1861,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): rabbitmq_num_queues = 2, rabbitmq_num_consumers = 2, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_queue_base = 'many_consumers', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1939,6 +1952,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size = 100, rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', rabbitmq_num_consumers = 2, @@ -2008,6 +2022,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): + logging.getLogger("pika").propagate = False instance.query( """ CREATE TABLE test.consumer_reconnect (key UInt64, value UInt64) @@ -2015,7 +2030,8 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'consumer_reconnect', rabbitmq_num_consumers = 10, - rabbitmq_flush_interval_ms=1000, + rabbitmq_flush_interval_ms = 100, + rabbitmq_max_block_size = 100, rabbitmq_num_queues = 10, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -2070,10 +2086,11 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): # revive_rabbitmq() while True: - result = instance.query("SELECT count(DISTINCT key) FROM test.view") - time.sleep(1) + result = instance.query("SELECT count(DISTINCT key) FROM test.view").strip() if int(result) == messages_num: break + print(f"Result: {result} / {messages_num}") + time.sleep(1) instance.query( """ @@ -2088,6 +2105,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): + logging.getLogger("pika").propagate = False instance.query( """ CREATE TABLE test.rabbitmq (key UInt64, value UInt64) @@ -2195,6 +2213,7 @@ def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_num_consumers = '5', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree @@ -2423,12 +2442,11 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): time.sleep(5) - result = instance.query( - "SELECT count() FROM test.rabbitmq_settings", ignore_error=True - ) - while int(result) != 10: - time.sleep(0.5) + while True: result = instance.query("SELECT count() FROM test.view", ignore_error=True) + if int(result) == 10: + break + time.sleep(0.5) instance.query("DROP TABLE test.rabbitmq_settings") @@ -2696,7 +2714,8 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): rabbitmq_check_result(result, True) - instance.query("DROP VIEW test.consumer") + instance.query("DROP VIEW test.consumer NO DELAY") + time.sleep(10) for i in range(50, 60): channel.basic_publish( exchange="mv", routing_key="", body=json.dumps({"key": i, "value": i}) From ab458e68cdc1f37d502d29392c2bcd52a4fc497b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Dec 2022 12:24:20 +0100 Subject: [PATCH 03/78] Fix tests --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 3 +-- .../integration/test_storage_rabbitmq/test.py | 22 +++++++++++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index b997e74abfc..8ec1f4c556b 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -173,12 +173,11 @@ Chunk RabbitMQSource::generateImpl() } total_rows += new_rows; + buffer->allowNext(); } if (total_rows >= max_block_size || buffer->isConsumerStopped() || isTimeLimitExceeded()) break; - - buffer->allowNext(); } LOG_TEST( diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index cc86f63c2e3..ea0fd99af6c 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -172,6 +172,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_commit_on_select = 1, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' """.format( @@ -225,6 +226,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): rabbitmq_commit_on_select = 1, rabbitmq_format = 'CSV', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_row_delimiter = '\\n'; """ ) @@ -267,6 +269,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_format = 'TSV', rabbitmq_commit_on_select = 1, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_queue_base = 'tsv', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -309,6 +312,7 @@ def test_rabbitmq_macros(rabbitmq_cluster): SETTINGS rabbitmq_host_port = '{rabbitmq_host}:{rabbitmq_port}', rabbitmq_commit_on_select = 1, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_name = '{rabbitmq_exchange_name}', rabbitmq_format = '{rabbitmq_format}' """ @@ -349,6 +353,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): rabbitmq_exchange_name = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() @@ -406,6 +411,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mvsq', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -450,6 +456,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'mmv', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view1 (key UInt64, value UInt64) @@ -508,6 +515,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): rabbitmq_exchange_name = 'pb', rabbitmq_format = 'Protobuf', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() @@ -583,6 +591,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'big', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value String) ENGINE = MergeTree @@ -800,6 +809,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'insert', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert1', rabbitmq_format = 'TSV', @@ -858,6 +868,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): rabbitmq_exchange_name = 'insert_headers', rabbitmq_exchange_type = 'headers', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_routing_key_list = 'test=insert,topic=headers', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -925,6 +936,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.rabbitmq_consume (key UInt64, value UInt64) @@ -934,6 +946,7 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'insert2', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; """ @@ -1115,6 +1128,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_name = 'direct_exchange_testing', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'direct_{0}', @@ -1207,6 +1221,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_routing_key_list = 'key_{0}', rabbitmq_exchange_name = 'fanout_exchange_testing', rabbitmq_exchange_type = 'fanout', @@ -1294,6 +1309,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.{0}', @@ -1318,6 +1334,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): rabbitmq_num_consumers = 2, rabbitmq_num_queues = 2, rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_exchange_name = 'topic_exchange_testing', rabbitmq_exchange_type = 'topic', rabbitmq_routing_key_list = '*.logs', @@ -1518,6 +1535,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.bindings_mv TO test.destination AS @@ -1608,6 +1626,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_exchange_name = 'headers_exchange_testing', rabbitmq_exchange_type = 'headers', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2020', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; @@ -1633,6 +1652,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2019', rabbitmq_format = 'JSONEachRow', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_row_delimiter = '\\n'; CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS SELECT key, value FROM test.headers_exchange_{0}; @@ -1706,6 +1726,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT value, key, _exchange_name, _channel_id, _delivery_tag, _redelivered FROM test.rabbitmq_virtuals; @@ -1775,6 +1796,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'virtuals_mv', rabbitmq_flush_interval_ms=1000, + rabbitmq_max_block_size=100, rabbitmq_format = 'JSONEachRow'; CREATE TABLE test.view (key UInt64, value UInt64, exchange_name String, channel_id String, delivery_tag UInt64, redelivered UInt8) ENGINE = MergeTree() From 8f1145f6e40794568637be15a41762fda5096cab Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 23 Dec 2022 11:47:51 +0100 Subject: [PATCH 04/78] Fix tests --- tests/integration/test_storage_rabbitmq/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index ea0fd99af6c..5511196fe1d 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -627,7 +627,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'test_sharding', - rabbitmq_num_queues = 10, + rabbitmq_num_queues = 5, rabbitmq_num_consumers = 10, rabbitmq_max_block_size = 100, rabbitmq_flush_interval_ms=10000, @@ -671,7 +671,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): connection.close() threads = [] - threads_num = 20 + threads_num = 10 for _ in range(threads_num): threads.append(threading.Thread(target=produce)) @@ -696,7 +696,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num ), "ClickHouse lost some messages: {}".format(result) - assert int(result2) == 10 + assert int(result2) == 5 def test_rabbitmq_mv_combo(rabbitmq_cluster): @@ -1025,10 +1025,10 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_name = 'over', rabbitmq_queue_base = 'over', rabbitmq_exchange_type = 'direct', - rabbitmq_num_consumers = 5, + rabbitmq_num_consumers = 3, rabbitmq_flush_interval_ms=1000, rabbitmq_max_block_size = 100, - rabbitmq_num_queues = 10, + rabbitmq_num_queues = 2, rabbitmq_routing_key_list = 'over', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; From dd6a2c2d3759a663cf9fdbc1d1b5e477a4fca8d1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 6 Jan 2023 16:53:24 +0100 Subject: [PATCH 05/78] Fix style check --- .../integration/test_storage_rabbitmq/test.py | 236 +++++++++--------- 1 file changed, 118 insertions(+), 118 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 299b59ad374..6ec89ccaa9b 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3296,8 +3296,8 @@ def test_block_based_formats_2(rabbitmq_cluster): def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): - instance.query( - """ + instance.query( + """ DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; @@ -3317,146 +3317,146 @@ def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.rabbitmq; """ - ) + ) - cancel = threading.Event() + cancel = threading.Event() - def produce(): - credentials = pika.PlainCredentials("root", "clickhouse") - parameters = pika.ConnectionParameters( - rabbitmq_cluster.rabbitmq_ip, - rabbitmq_cluster.rabbitmq_port, - "/", - credentials, - ) - connection = pika.BlockingConnection(parameters) + def produce(): + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + connection = pika.BlockingConnection(parameters) - while not cancel.is_set(): - try: - channel = connection.channel() - channel.basic_publish( - exchange="flush_by_block", - routing_key="", - body=json.dumps({"key": 0, "value": 0}), - ) - except e: - print(f"Got error: {str(e)}") + while not cancel.is_set(): + try: + channel = connection.channel() + channel.basic_publish( + exchange="flush_by_block", + routing_key="", + body=json.dumps({"key": 0, "value": 0}), + ) + except e: + print(f"Got error: {str(e)}") - produce_thread = threading.Thread(target=produce) - produce_thread.start() + produce_thread = threading.Thread(target=produce) + produce_thread.start() - while 0 == int( - instance.query( - "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" - ) - ): - time.sleep(0.5) + while 0 == int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ): + time.sleep(0.5) - cancel.set() - produce_thread.join() + cancel.set() + produce_thread.join() - # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). - result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # logging.debug(result) + # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). + result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") + # logging.debug(result) - instance.query( - """ + instance.query( + """ DROP TABLE test.consumer; DROP TABLE test.view; DROP TABLE test.rabbitmq; """ - ) + ) - # 100 = first poll should return 100 messages (and rows) - # not waiting for stream_flush_interval_ms - assert ( - int(result) == 100 - ), "Messages from rabbitmq should be flushed when block of size rabbitmq_max_block_size is formed!" + # 100 = first poll should return 100 messages (and rows) + # not waiting for stream_flush_interval_ms + assert ( + int(result) == 100 + ), "Messages from rabbitmq should be flushed when block of size rabbitmq_max_block_size is formed!" - def test_rabbitmq_flush_by_time(rabbitmq_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; +def test_rabbitmq_flush_by_time(rabbitmq_cluster): + instance.query( + """ + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; - CREATE TABLE test.rabbitmq (key UInt64, value UInt64) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'flush_by_time', - rabbitmq_queue_base = 'flush_by_time', - rabbitmq_max_block_size = 100, - rabbitmq_flush_interval_ms = 5000, - rabbitmq_format = 'JSONEachRow'; + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_exchange_name = 'flush_by_time', + rabbitmq_queue_base = 'flush_by_time', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 5000, + rabbitmq_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) - ENGINE = MergeTree() - ORDER BY key; - """ - ) + CREATE TABLE test.view (key UInt64, value UInt64, ts DateTime64(3) MATERIALIZED now64(3)) + ENGINE = MergeTree() + ORDER BY key; + """ + ) - cancel = threading.Event() + cancel = threading.Event() - def produce(): - credentials = pika.PlainCredentials("root", "clickhouse") - parameters = pika.ConnectionParameters( - rabbitmq_cluster.rabbitmq_ip, - rabbitmq_cluster.rabbitmq_port, - "/", - credentials, - ) - connection = pika.BlockingConnection(parameters) + def produce(): + credentials = pika.PlainCredentials("root", "clickhouse") + parameters = pika.ConnectionParameters( + rabbitmq_cluster.rabbitmq_ip, + rabbitmq_cluster.rabbitmq_port, + "/", + credentials, + ) + connection = pika.BlockingConnection(parameters) - while not cancel.is_set(): - try: - channel = connection.channel() - channel.basic_publish( - exchange="flush_by_time", - routing_key="", - body=json.dumps({"key": 0, "value": 0}), - ) - print("Produced a message") - time.sleep(0.8) - except e: - print(f"Got error: {str(e)}") + while not cancel.is_set(): + try: + channel = connection.channel() + channel.basic_publish( + exchange="flush_by_time", + routing_key="", + body=json.dumps({"key": 0, "value": 0}), + ) + print("Produced a message") + time.sleep(0.8) + except e: + print(f"Got error: {str(e)}") - produce_thread = threading.Thread(target=produce) - produce_thread.start() + produce_thread = threading.Thread(target=produce) + produce_thread.start() - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - """ - ) + instance.query( + """ + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq; + """ + ) - while True: - time.sleep(0.2) - count = instance.query( - "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view'" - ) - print(f"kssenii total count: {count}") - count = int( - instance.query( - "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" - ) - ) - print(f"kssenii count: {count}") - if count > 0: - break + while True: + time.sleep(0.2) + count = instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view'" + ) + print(f"kssenii total count: {count}") + count = int( + instance.query( + "SELECT count() FROM system.parts WHERE database = 'test' AND table = 'view' AND name = 'all_1_1_0'" + ) + ) + print(f"kssenii count: {count}") + if count > 0: + break - time.sleep(12) - result = instance.query("SELECT uniqExact(ts) FROM test.view") + time.sleep(12) + result = instance.query("SELECT uniqExact(ts) FROM test.view") - cancel.set() - produce_thread.join() + cancel.set() + produce_thread.join() - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - DROP TABLE test.rabbitmq; - """ - ) + instance.query( + """ + DROP TABLE test.consumer; + DROP TABLE test.view; + DROP TABLE test.rabbitmq; + """ + ) - assert int(result) == 3 + assert int(result) == 3 From f80246644ccd146ad2cecba985bf2a8e45a72959 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Jan 2023 18:04:09 +0100 Subject: [PATCH 06/78] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index be066818c94..bc7e633dfb3 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1,10 +1,5 @@ import pytest -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/45160 - -pytestmark = pytest.mark.skip - import json import os.path as p import random From 1452d2588ce0cdc984dd6ec9161f34eb24475e25 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Jan 2023 18:06:43 +0100 Subject: [PATCH 07/78] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bc7e633dfb3..b4e7c98e282 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -631,7 +631,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_num_queues = 5, rabbitmq_num_consumers = 10, rabbitmq_max_block_size = 100, - rabbitmq_flush_interval_ms=10000, + rabbitmq_flush_interval_ms=500, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) @@ -1974,7 +1974,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): CREATE TABLE test.consume (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_flush_interval_ms=1000, + rabbitmq_flush_interval_ms=500, rabbitmq_max_block_size = 100, rabbitmq_exchange_name = 'producer_reconnect', rabbitmq_format = 'JSONEachRow', From 2fb49503dacee99091bb3804a71006394440e125 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Thu, 19 Jan 2023 14:02:06 +0800 Subject: [PATCH 08/78] Make addAttribute exception safe --- src/Common/OpenTelemetryTraceContext.cpp | 71 ++++++++++-------------- src/Common/OpenTelemetryTraceContext.h | 19 +++---- 2 files changed, 39 insertions(+), 51 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 515060803d6..365725c1a98 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -14,76 +14,65 @@ namespace OpenTelemetry thread_local TracingContextOnThread current_thread_trace_context; -void Span::addAttribute(std::string_view name, UInt64 value) +bool Span::addAttribute(std::string_view name, UInt64 value) noexcept { if (!this->isTraceEnabled() || name.empty()) - return; + return false; - this->attributes.push_back(Tuple{name, toString(value)}); + return addAttribute(name, toString(value)); } -void Span::addAttributeIfNotZero(std::string_view name, UInt64 value) +bool Span::addAttributeIfNotZero(std::string_view name, UInt64 value) noexcept { - if (value != 0) - addAttribute(name, value); + if (!this->isTraceEnabled() || name.empty() || value == 0) + return false; + + return addAttribute(name, toString(value)); } -void Span::addAttribute(std::string_view name, std::string_view value) +bool Span::addAttribute(std::string_view name, std::string_view value) noexcept { if (!this->isTraceEnabled() || name.empty()) - return; + return false; - this->attributes.push_back(Tuple{name, value}); + try + { + this->attributes.push_back(Tuple{name, value}); + } + catch (...) + { + return false; + } + return true; } -void Span::addAttributeIfNotEmpty(std::string_view name, std::string_view value) +bool Span::addAttributeIfNotEmpty(std::string_view name, std::string_view value) noexcept { - if (!this->isTraceEnabled() || name.empty() || value.empty()) - return; - - this->attributes.push_back(Tuple{name, value}); + return value.empty() ? false : addAttribute(name, value); } -void Span::addAttribute(std::string_view name, std::function value_supplier) +bool Span::addAttribute(std::string_view name, std::function value_supplier) noexcept { if (!this->isTraceEnabled() || !value_supplier) - return; + return false; - String value = value_supplier(); - if (value.empty()) - return; - - this->attributes.push_back(Tuple{name, value}); + return addAttributeIfNotEmpty(name, value_supplier()); } -void Span::addAttribute(const Exception & e) noexcept +bool Span::addAttribute(const Exception & e) noexcept { if (!this->isTraceEnabled()) - return; + return false; - try - { - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); - } - catch (...) - { - /// Ignore exceptions - } + return addAttribute("clickhouse.exception", getExceptionMessage(e, false)); } -void Span::addAttribute(std::exception_ptr e) noexcept +bool Span::addAttribute(std::exception_ptr e) noexcept { if (!this->isTraceEnabled() || e == nullptr) - return; + return false; - try - { - this->attributes.push_back(Tuple{"clickhouse.exception", getExceptionMessage(e, false)}); - } - catch (...) - { - /// Ignore exceptions - } + return addAttribute("clickhouse.exception", getExceptionMessage(e, false)); } SpanHolder::SpanHolder(std::string_view _operation_name) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 03bac2891fc..930262f2c92 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -23,16 +23,15 @@ struct Span UInt64 finish_time_us = 0; Map attributes; - void addAttribute(std::string_view name, UInt64 value); - void addAttributeIfNotZero(std::string_view name, UInt64 value); - void addAttribute(std::string_view name, std::string_view value); - void addAttributeIfNotEmpty(std::string_view name, std::string_view value); - void addAttribute(std::string_view name, std::function value_supplier); - - /// Following two methods are declared as noexcept to make sure they're exception safe - /// This is because they're usually called in exception handler - void addAttribute(const Exception & e) noexcept; - void addAttribute(std::exception_ptr e) noexcept; + /// Following methods are declared as noexcept to make sure they're exception safe + /// This is because sometimes they will be called in exception handlers/dtor + bool addAttribute(std::string_view name, UInt64 value) noexcept; + bool addAttributeIfNotZero(std::string_view name, UInt64 value) noexcept; + bool addAttribute(std::string_view name, std::string_view value) noexcept; + bool addAttributeIfNotEmpty(std::string_view name, std::string_view value) noexcept; + bool addAttribute(std::string_view name, std::function value_supplier) noexcept; + bool addAttribute(const Exception & e) noexcept; + bool addAttribute(std::exception_ptr e) noexcept; bool isTraceEnabled() const { From 75191fd86853f7b656d6c868bd0c1ed52c398438 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 14:51:44 +0800 Subject: [PATCH 09/78] Catch exception raised by function object --- src/Common/OpenTelemetryTraceContext.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 365725c1a98..65e66b1da96 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -56,7 +56,15 @@ bool Span::addAttribute(std::string_view name, std::function value_sup if (!this->isTraceEnabled() || !value_supplier) return false; - return addAttributeIfNotEmpty(name, value_supplier()); + try + { + return addAttributeIfNotEmpty(name, value_supplier()); + } + catch (...) + { + /// Ignore exception raised by value_supplier + return false; + } } bool Span::addAttribute(const Exception & e) noexcept From fb8452aa4909058962f2ddc055fe3e029fee44f3 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 13 Jan 2023 11:26:41 +0800 Subject: [PATCH 10/78] Add last_exception_time to replication_queue --- src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp | 1 + src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 1 + src/Storages/System/StorageSystemReplicationQueue.cpp | 4 +++- 4 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 36a5957bf1d..de416c3388a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -91,6 +91,7 @@ bool ReplicatedMergeMutateTaskBase::executeStep() auto & log_entry = selected_entry->log_entry; log_entry->exception = saved_exception; + log_entry->last_exception_time = time(nullptr); if (log_entry->type == ReplicatedMergeTreeLogEntryData::MUTATE_PART) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 84115cb2b8a..476d4797eef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -157,6 +157,7 @@ struct ReplicatedMergeTreeLogEntryData /// Access under queue_mutex, see ReplicatedMergeTreeQueue. size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. + time_t last_exception_time = 0; /// The time at which the last exception occurred. time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. size_t num_postponed = 0; /// The number of times the action was postponed. String postpone_reason; /// The reason why the action was postponed, if it was postponed. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index bd75d76109a..628b3048cfc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1691,6 +1691,7 @@ bool ReplicatedMergeTreeQueue::processEntry( { std::lock_guard lock(state_mutex); entry->exception = saved_exception; + entry->last_exception_time = time(nullptr); return false; } diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 8acd192eac4..fd9f874052d 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -38,6 +38,7 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() { "is_currently_executing", std::make_shared() }, { "num_tries", std::make_shared() }, { "last_exception", std::make_shared() }, + { "last_exception_time", std::make_shared() }, { "last_attempt_time", std::make_shared() }, { "num_postponed", std::make_shared() }, { "postpone_reason", std::make_shared() }, @@ -141,7 +142,8 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, Conte res_columns[col_num++]->insert(entry.detach); res_columns[col_num++]->insert(entry.currently_executing); res_columns[col_num++]->insert(entry.num_tries); - res_columns[col_num++]->insert(entry.exception ? getExceptionMessage(entry.exception, false) : ""); + res_columns[col_num++]->insert(entry.exception ? getExceptionMessage(entry.exception, true) : ""); + res_columns[col_num++]->insert(UInt64(entry.last_exception_time)); res_columns[col_num++]->insert(UInt64(entry.last_attempt_time)); res_columns[col_num++]->insert(entry.num_postponed); res_columns[col_num++]->insert(entry.postpone_reason); From 1fe9bde59409db16bc2da8f720877986899d9111 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 11:04:36 +0800 Subject: [PATCH 11/78] Set replica name to zero copy exclusive lock --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 7 ++++--- src/Storages/MergeTree/ZeroCopyLock.cpp | 4 ++-- src/Storages/MergeTree/ZeroCopyLock.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.h | 2 +- 5 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index bd75d76109a..09b9365cf71 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1363,10 +1363,11 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (!disk->supportZeroCopyReplication()) only_s3_storage = false; - if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0])) + String replica_to_execute_merge; + if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge)) { - constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting other replica to execute it and will fetch after."; - out_postpone_reason = fmt::format(fmt_string, entry.new_part_name); + constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after."; + out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge;); return false; } } diff --git a/src/Storages/MergeTree/ZeroCopyLock.cpp b/src/Storages/MergeTree/ZeroCopyLock.cpp index dbb12d0d610..53dfe0c769f 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.cpp +++ b/src/Storages/MergeTree/ZeroCopyLock.cpp @@ -2,8 +2,8 @@ namespace DB { - ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path) - : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", "")) + ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message) + : lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", lock_message)) { } } diff --git a/src/Storages/MergeTree/ZeroCopyLock.h b/src/Storages/MergeTree/ZeroCopyLock.h index 96709fb01c9..4e73b27804c 100644 --- a/src/Storages/MergeTree/ZeroCopyLock.h +++ b/src/Storages/MergeTree/ZeroCopyLock.h @@ -12,7 +12,7 @@ namespace DB /// because due to bad abstraction we use it in MergeTreeData. struct ZeroCopyLock { - ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path); + ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message); /// Actual lock std::unique_ptr lock; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d4fd329ef90..320e3754b83 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8453,14 +8453,14 @@ Strings StorageReplicatedMergeTree::getZeroCopyPartPath( return res; } -bool StorageReplicatedMergeTree::checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk) +bool StorageReplicatedMergeTree::checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk, String & lock_replica) { auto path = getZeroCopyPartPath(part_name, disk); if (path) { /// FIXME auto lock_path = fs::path(*path) / "part_exclusive_lock"; - if (getZooKeeper()->exists(lock_path)) + if (getZooKeeper()->tryGet(lock_path, lock_replica)) { return true; } @@ -8493,7 +8493,7 @@ std::optional StorageReplicatedMergeTree::tryCreateZeroCopyExclusi zookeeper->createIfNotExists(zc_zookeeper_path, ""); /// Create actual lock - ZeroCopyLock lock(zookeeper, zc_zookeeper_path); + ZeroCopyLock lock(zookeeper, zc_zookeeper_path, replica_name); if (lock.lock->tryLock()) return lock; else diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 71f143aa8a4..842ff584a8a 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -858,7 +858,7 @@ private: // Create table id if needed void createTableSharedID() const; - bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk); + bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk, String & lock_replica); std::optional getZeroCopyPartPath(const String & part_name, const DiskPtr & disk); From be8ceeb4bb857028c8bba7076e65dc33dbdd5fce Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 13 Jan 2023 18:42:52 +0800 Subject: [PATCH 12/78] Save stack trace in part_log --- src/Common/Exception.cpp | 6 ++++++ src/Common/Exception.h | 2 ++ src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 11 files changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 35231354651..b83ddf09c7f 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -546,6 +546,12 @@ ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_ return ExecutionStatus(getCurrentExceptionCode(), msg); } +ExecutionStatus ExecutionStatus::fromCurrentExceptionWithStackTrace(const std::string & start_of_message) +{ + String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(true, true); + return ExecutionStatus(getCurrentExceptionCode(), msg); +} + ExecutionStatus ExecutionStatus::fromText(const std::string & data) { ExecutionStatus status; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 0f459a887f1..58da20359ad 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -204,6 +204,8 @@ struct ExecutionStatus : code(return_code), message(exception_message) {} static ExecutionStatus fromCurrentException(const std::string & start_of_message = ""); + + static ExecutionStatus fromCurrentExceptionWithStackTrace(const std::string & start_of_message = ""); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 97226825629..ca3e25ef077 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index cc5e87956a1..c38fefd9d32 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b1384dc799f..e1e1312cd6c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index b432841d5b0..65b3f8b15a8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index b1714076a46..28452aaf856 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 36a5957bf1d..09fda943600 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException()); + part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException()); + part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 14ae620176a..9ebed2467ad 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c8dc53f5036..ec17c765e05 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d4fd329ef90..04fbdad22d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException()); + write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 38799eb85238003b15f6a0e397a3e0650f0aa9de Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 16:35:28 +0800 Subject: [PATCH 13/78] Merge two methods as one --- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 9 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index ca3e25ef077..16efde3e4cc 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index c38fefd9d32..9f24839f1e1 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e1e1312cd6c..6184669b223 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 65b3f8b15a8..8b5c6bf013d 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 28452aaf856..e801c0a8879 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 09fda943600..7225a7c41d3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9ebed2467ad..c4a8f837ec9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ec17c765e05..5536e8af793 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 04fbdad22d1..c0d0780a550 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentExceptionWithStackTrace()); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentExceptionWithStackTrace()); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From f867c4c2061858bc2a3e58855acaa90b93e753dc Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 20 Jan 2023 16:43:10 +0800 Subject: [PATCH 14/78] Add an overriden fromCurrentException to simply the code --- src/Common/Exception.cpp | 9 ++++----- src/Common/Exception.h | 5 ++--- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 11 files changed, 22 insertions(+), 24 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index b83ddf09c7f..10fc3dcbfae 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -540,15 +540,14 @@ bool ExecutionStatus::tryDeserializeText(const std::string & data) return true; } -ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message) +ExecutionStatus ExecutionStatus::fromCurrentException(bool with_stacktrace) { - String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(false, true); - return ExecutionStatus(getCurrentExceptionCode(), msg); + return ExecutionStatus(getCurrentExceptionCode(), getCurrentExceptionMessage(with_stacktrace, true)); } -ExecutionStatus ExecutionStatus::fromCurrentExceptionWithStackTrace(const std::string & start_of_message) +ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message, bool with_stacktrace) { - String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(true, true); + String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(with_stacktrace, true); return ExecutionStatus(getCurrentExceptionCode(), msg); } diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 58da20359ad..96668c68cd3 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -203,9 +203,8 @@ struct ExecutionStatus explicit ExecutionStatus(int return_code, const std::string & exception_message = "") : code(return_code), message(exception_message) {} - static ExecutionStatus fromCurrentException(const std::string & start_of_message = ""); - - static ExecutionStatus fromCurrentExceptionWithStackTrace(const std::string & start_of_message = ""); + static ExecutionStatus fromCurrentException(bool with_stacktrace = false); + static ExecutionStatus fromCurrentException(const std::string & start_of_message, bool with_stacktrace = false); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 16efde3e4cc..b3934098d0d 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -333,7 +333,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 9f24839f1e1..e59f7fda44c 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6184669b223..5addb24b0ca 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7480,7 +7480,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 8b5c6bf013d..7bf3a95876b 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -209,7 +209,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index e801c0a8879..1ab1e2a82ec 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -101,7 +101,7 @@ bool MutatePlainMergeTreeTask::executeStep() String exception_message = getCurrentExceptionMessage(false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message); - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 7225a7c41d3..bbe83814497 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException("", true)); + part_log_writer(ExecutionStatus::fromCurrentException(true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException("", true)); + part_log_writer(ExecutionStatus::fromCurrentException(true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c4a8f837ec9..03b2dadde5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5536e8af793..20b0c16edb7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1784,7 +1784,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } @@ -1864,7 +1864,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c0d0780a550..82ff0a0bd83 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2336,7 +2336,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4185,7 +4185,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } @@ -4296,7 +4296,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException("", true)); + write_part_log(ExecutionStatus::fromCurrentException(true)); throw; } @@ -7029,7 +7029,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7255,7 +7255,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From f39cd0b3927b690a0214f0c46d1d8ac14abd79c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 27 Jan 2023 17:22:24 +0100 Subject: [PATCH 15/78] Update tests --- tests/integration/test_storage_rabbitmq/test.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index be066818c94..a3c467fb020 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1,10 +1,5 @@ import pytest -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/45160 - -pytestmark = pytest.mark.skip - import json import os.path as p import random @@ -702,7 +697,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): assert ( int(result1) == messages_num * threads_num ), "ClickHouse lost some messages: {}".format(result) - assert int(result2) == 5 + assert int(result2) == 10 def test_rabbitmq_mv_combo(rabbitmq_cluster): @@ -2049,6 +2044,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ) +@pytest.mark.skip(reason="Timeout: FIXME") def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): logging.getLogger("pika").propagate = False instance.query( @@ -3099,6 +3095,8 @@ def test_row_based_formats(rabbitmq_cluster): rabbitmq_format = '{format_name}', rabbitmq_exchange_name = '{format_name}', rabbitmq_exchange_type = 'direct', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 1000, rabbitmq_routing_key_list = '{format_name}', rabbitmq_max_rows_per_message = 5; @@ -3167,6 +3165,8 @@ def test_block_based_formats_1(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'PrettySpace', rabbitmq_exchange_type = 'direct', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 1000, rabbitmq_routing_key_list = 'PrettySpace', rabbitmq_format = 'PrettySpace'; """ @@ -3243,6 +3243,8 @@ def test_block_based_formats_2(rabbitmq_cluster): rabbitmq_format = '{format_name}', rabbitmq_exchange_name = '{format_name}', rabbitmq_exchange_type = 'direct', + rabbitmq_max_block_size = 100, + rabbitmq_flush_interval_ms = 1000, rabbitmq_routing_key_list = '{format_name}'; CREATE MATERIALIZED VIEW test.view Engine=Log AS From 839cd614fbf00ea5ba0808d346fbe1d7fb3c9d6d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 30 Jan 2023 19:00:48 +0000 Subject: [PATCH 16/78] fix memory leak in azure sdk --- contrib/azure | 2 +- programs/local/CMakeLists.txt | 3 +++ programs/local/LocalServer.cpp | 12 ++++++++++++ programs/server/CMakeLists.txt | 3 +++ programs/server/Server.cpp | 17 +++++++++++++++++ src/Common/ThreadPool.cpp | 20 ++++++++++++++++++++ src/Common/ThreadPool.h | 16 +++++++++++++++- 7 files changed, 71 insertions(+), 2 deletions(-) diff --git a/contrib/azure b/contrib/azure index ea8c3044f43..e4fcdfc81e3 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea8c3044f43f5afa7016d2d580ed201f495d7e94 +Subproject commit e4fcdfc81e337e589ce231a452dcc280fcbb3f99 diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index 6943af48ab9..565b67d0020 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -19,6 +19,9 @@ target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib) if (TARGET ch_rust::skim) target_link_libraries(clickhouse-local-lib PRIVATE ch_rust::skim) endif() +if (TARGET ch_contrib::azure_sdk) + target_link_libraries(clickhouse-local-lib PRIVATE ch_contrib::azure_sdk) +endif() # Always use internal readpassphrase target_link_libraries(clickhouse-local-lib PRIVATE readpassphrase) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2f0f98ae857..133d629bbb1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -51,6 +51,10 @@ #include #endif +#if USE_AZURE_BLOB_STORAGE +# include +#endif + namespace fs = std::filesystem; @@ -115,6 +119,14 @@ void LocalServer::initialize(Poco::Util::Application & self) config().getUInt("thread_pool_queue_size", 10000) ); +#if USE_AZURE_BLOB_STORAGE + /// See the explanation near the same line in Server.cpp + GlobalThreadPool::instance().addOnDestroyCallback([] + { + Azure::Storage::_internal::XmlGlobalDeinitialize(); + }); +#endif + IOThreadPool::initialize( config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 2cfa748d585..855973d10e1 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -27,6 +27,9 @@ set (CLICKHOUSE_SERVER_LINK if (TARGET ch_contrib::jemalloc) list(APPEND CLICKHOUSE_SERVER_LINK PRIVATE ch_contrib::jemalloc) endif() +if (TARGET ch_contrib::azure_sdk) + list(APPEND CLICKHOUSE_SERVER_LINK PRIVATE ch_contrib::azure_sdk) +endif() clickhouse_program_add(server) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8b6c43b450e..74071edc289 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -128,6 +128,10 @@ # include #endif +#if USE_AZURE_BLOB_STORAGE +# include +#endif + namespace CurrentMetrics { extern const Metric Revision; @@ -750,6 +754,19 @@ try config().getUInt("max_thread_pool_free_size", 1000), config().getUInt("thread_pool_queue_size", 10000)); +#if USE_AZURE_BLOB_STORAGE + /// It makes sense to deinitialize libxml after joining of all threads + /// in global pool because libxml uses thread-local memory allocations via + /// 'pthread_key_create' and 'pthread_setspecific' which should be deallocated + /// at 'pthread_exit'. Deinitialization of libxml leads to call of 'pthread_key_delete' + /// and if it is done before joining of threads, allocated memory will not be freed + /// and there may be memory leaks in threads that used libxml. + GlobalThreadPool::instance().addOnDestroyCallback([] + { + Azure::Storage::_internal::XmlGlobalDeinitialize(); + }); +#endif + IOThreadPool::initialize( config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index e65b5511e05..2843c4b1ad9 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -209,6 +210,7 @@ ThreadPoolImpl::~ThreadPoolImpl() /// and the destruction order of global variables is unspecified. finalize(); + onDestroy(); } template @@ -227,6 +229,24 @@ void ThreadPoolImpl::finalize() threads.clear(); } +template +void ThreadPoolImpl::addOnDestroyCallback(OnDestroyCallback && callback) +{ + std::lock_guard lock(mutex); + on_destroy_callbacks.push(std::move(callback)); +} + +template +void ThreadPoolImpl::onDestroy() +{ + while (!on_destroy_callbacks.empty()) + { + auto callback = std::move(on_destroy_callbacks.top()); + on_destroy_callbacks.pop(); + NOEXCEPT_SCOPE({ callback(); }); + } +} + template size_t ThreadPoolImpl::active() const { diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index f19a412db37..9964a84924e 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -9,6 +9,7 @@ #include #include #include +#include #include @@ -80,6 +81,16 @@ public: void setQueueSize(size_t value); size_t getMaxThreads() const; + /// Adds a callback which is called in destrutor after + /// joining of all threads. The order of calling callbacks + /// is reversed to the order of their addition. + /// It may be useful for static thread pools to call + /// function after joining of threads because order + /// of destructors of global static objects and callbacks + /// added by atexit is undefined for different translation units. + using OnDestroyCallback = std::function; + void addOnDestroyCallback(OnDestroyCallback && callback); + private: mutable std::mutex mutex; std::condition_variable job_finished; @@ -111,6 +122,7 @@ private: boost::heap::priority_queue jobs; std::list threads; std::exception_ptr first_exception; + std::stack on_destroy_callbacks; template ReturnType scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); @@ -118,6 +130,7 @@ private: void worker(typename std::list::iterator thread_it); void finalize(); + void onDestroy(); }; @@ -146,7 +159,8 @@ class GlobalThreadPool : public FreeThreadPool, private boost::noncopyable size_t queue_size_, const bool shutdown_on_exception_) : FreeThreadPool(max_threads_, max_free_threads_, queue_size_, shutdown_on_exception_) - {} + { + } public: static void initialize(size_t max_threads = 10000, size_t max_free_threads = 1000, size_t queue_size = 10000); From 31e8b692f4b7735ef107d976e27713a5bcbe03af Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 31 Jan 2023 01:13:23 +0000 Subject: [PATCH 17/78] fix typo --- src/Common/ThreadPool.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 9964a84924e..18be05f6c71 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -81,7 +81,7 @@ public: void setQueueSize(size_t value); size_t getMaxThreads() const; - /// Adds a callback which is called in destrutor after + /// Adds a callback which is called in destructor after /// joining of all threads. The order of calling callbacks /// is reversed to the order of their addition. /// It may be useful for static thread pools to call From d1d5f0626001186504794c1850a42525ae422d51 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 31 Jan 2023 09:24:43 -0500 Subject: [PATCH 18/78] add libaccel-config as submodule and static link to QPL --- .gitmodules | 3 + contrib/CMakeLists.txt | 1 + contrib/idxd-config | 1 + contrib/idxd-config-cmake/CMakeLists.txt | 26 +++ .../idxd-config-cmake/idxd-header/config.h | 159 ++++++++++++++++++ contrib/idxd-config-cmake/uuid/uuid.h | 4 + contrib/qpl-cmake/CMakeLists.txt | 7 +- 7 files changed, 195 insertions(+), 6 deletions(-) create mode 160000 contrib/idxd-config create mode 100644 contrib/idxd-config-cmake/CMakeLists.txt create mode 100644 contrib/idxd-config-cmake/idxd-header/config.h create mode 100644 contrib/idxd-config-cmake/uuid/uuid.h diff --git a/.gitmodules b/.gitmodules index b4673f113b7..ab169cf9f77 100644 --- a/.gitmodules +++ b/.gitmodules @@ -330,3 +330,6 @@ [submodule "contrib/crc32-vpmsum"] path = contrib/crc32-vpmsum url = https://github.com/antonblanchard/crc32-vpmsum.git +[submodule "contrib/idxd-config"] + path = contrib/idxd-config + url = https://github.com/intel/idxd-config.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f5d1315cc02..180e81b7641 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -178,6 +178,7 @@ add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (c-ares-cmake c-ares) add_contrib (qpl-cmake qpl) +add_contrib (idxd-config-cmake idxd-config) add_contrib (morton-nd-cmake morton-nd) if (ARCH_S390X) diff --git a/contrib/idxd-config b/contrib/idxd-config new file mode 160000 index 00000000000..f6605c41a73 --- /dev/null +++ b/contrib/idxd-config @@ -0,0 +1 @@ +Subproject commit f6605c41a735e3fdfef2d2d18655a33af6490b99 diff --git a/contrib/idxd-config-cmake/CMakeLists.txt b/contrib/idxd-config-cmake/CMakeLists.txt new file mode 100644 index 00000000000..ee981d0a57b --- /dev/null +++ b/contrib/idxd-config-cmake/CMakeLists.txt @@ -0,0 +1,26 @@ +## Utility library for controlling and configuring Intel® In-Memory Analytics Accelerator (Intel® IAA) available on Next Generation Intel® Xeon® Scalable processors. +set(LIBACCEL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config") +set(LIBACCEL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/idxd-config") +set(UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake") +set(LIBACCEL_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake/idxd-header") +#set(SYSROOT_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc/usr/include") +set(SRCS + "${LIBACCEL_SOURCE_DIR}/accfg/lib/libaccfg.c" + "${LIBACCEL_SOURCE_DIR}/util/log.c" + "${LIBACCEL_SOURCE_DIR}/util/sysfs.c" +) +#"${LIBACCEL_SOURCE_DIR}/util/log.c" +#"${LIBACCEL_SOURCE_DIR}/util/sysfs.c" +add_library(_LIBACCEL ${SRCS}) + +#target_link_libraries(_LIBACCEL PRIVATE ch_contrib::zlib) +target_compile_options(_LIBACCEL PRIVATE "-D_GNU_SOURCE") + +target_include_directories(_LIBACCEL BEFORE + PRIVATE ${UUID_DIR} + PRIVATE ${LIBACCEL_HEADER_DIR} + PRIVATE ${LIBACCEL_SOURCE_DIR}) + +#target_include_directories(_LIBACCEL BEFORE PUBLIC ${LIBACCEL_BINARY_DIR}) + +add_library(ch_contrib::LIBACCEL ALIAS _LIBACCEL) diff --git a/contrib/idxd-config-cmake/idxd-header/config.h b/contrib/idxd-config-cmake/idxd-header/config.h new file mode 100644 index 00000000000..e133ee1a56a --- /dev/null +++ b/contrib/idxd-config-cmake/idxd-header/config.h @@ -0,0 +1,159 @@ +/* config.h. Generated from config.h.in by configure. */ +/* config.h.in. Generated from configure.ac by autoheader. */ + +/* Define if building universal (internal helper macro) */ +/* #undef AC_APPLE_UNIVERSAL_BUILD */ + +/* Debug messages. */ +/* #undef ENABLE_DEBUG */ + +/* Documentation / man pages. */ +#define ENABLE_DOCS 1 + +/* System logging. */ +#define ENABLE_LOGGING 1 + +/* accfg test support */ +/* #undef ENABLE_TEST */ + +/* Define to 1 if big-endian-arch */ +/* #undef HAVE_BIG_ENDIAN */ + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_LINUX_VERSION_H 1 + +/* Define to 1 if little-endian-arch */ +#define HAVE_LITTLE_ENDIAN 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_MEMORY_H 1 + +/* Define to 1 if you have the `secure_getenv' function. */ +#define HAVE_SECURE_GETENV 1 + +/* Define to 1 if you have statement expressions. */ +#define HAVE_STATEMENT_EXPR 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if typeof works with your compiler. */ +#define HAVE_TYPEOF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to 1 if using libuuid */ +#define HAVE_UUID 1 + +/* Define to 1 if you have the `__secure_getenv' function. */ +/* #undef HAVE___SECURE_GETENV */ + +/* Define to the sub-directory where libtool stores uninstalled libraries. */ +#define LT_OBJDIR ".libs/" + +/* Name of package */ +#define PACKAGE "accel-config" + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "linux-dsa@lists.01.org" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "accel-config" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "accel-config 3.5.2.gitf6605c41" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "accel-config" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "https://github.com/xxx/accel-config" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "3.5.2.gitf6605c41" + +/* Define to 1 if you have the ANSI C header files. */ +#define STDC_HEADERS 1 + +/* Enable extensions on AIX 3, Interix. */ +#ifndef _ALL_SOURCE +# define _ALL_SOURCE 1 +#endif +/* Enable GNU extensions on systems that have them. */ +#ifndef _GNU_SOURCE +# define _GNU_SOURCE 1 +#endif +/* Enable threading extensions on Solaris. */ +#ifndef _POSIX_PTHREAD_SEMANTICS +# define _POSIX_PTHREAD_SEMANTICS 1 +#endif +/* Enable extensions on HP NonStop. */ +#ifndef _TANDEM_SOURCE +# define _TANDEM_SOURCE 1 +#endif +/* Enable general extensions on Solaris. */ +#ifndef __EXTENSIONS__ +# define __EXTENSIONS__ 1 +#endif + + +/* Version number of package */ +#define VERSION "3.5.2.gitf6605c41" + +/* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most + significant byte first (like Motorola and SPARC, unlike Intel). */ +#if defined AC_APPLE_UNIVERSAL_BUILD +# if defined __BIG_ENDIAN__ +# define WORDS_BIGENDIAN 1 +# endif +#else +# ifndef WORDS_BIGENDIAN +/* # undef WORDS_BIGENDIAN */ +# endif +#endif + +/* Enable large inode numbers on Mac OS X 10.5. */ +#ifndef _DARWIN_USE_64_BIT_INODE +# define _DARWIN_USE_64_BIT_INODE 1 +#endif + +/* Number of bits in a file offset, on hosts where this is settable. */ +/* #undef _FILE_OFFSET_BITS */ + +/* Define for large files, on AIX-style hosts. */ +/* #undef _LARGE_FILES */ + +/* Define to 1 if on MINIX. */ +/* #undef _MINIX */ + +/* Define to 2 if the system does not provide POSIX.1 features except with + this defined. */ +/* #undef _POSIX_1_SOURCE */ + +/* Define to 1 if you need to in order for `stat' and other things to work. */ +/* #undef _POSIX_SOURCE */ + +/* Define to __typeof__ if your compiler spells it that way. */ +/* #undef typeof */ diff --git a/contrib/idxd-config-cmake/uuid/uuid.h b/contrib/idxd-config-cmake/uuid/uuid.h new file mode 100644 index 00000000000..bf108ba0d29 --- /dev/null +++ b/contrib/idxd-config-cmake/uuid/uuid.h @@ -0,0 +1,4 @@ +#ifndef _QPL_UUID_UUID_H +#define _QPL_UUID_UUID_H +typedef unsigned char uuid_t[16]; +#endif /* _QPL_UUID_UUID_H */ diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index beef8432e7a..8ac2d7bdf9b 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -315,13 +315,8 @@ target_compile_definitions(_qpl PRIVATE -DQPL_BADARG_CHECK PUBLIC -DENABLE_QPL_COMPRESSION) -find_library(LIBACCEL accel-config) -if(NOT LIBACCEL) - message(FATAL_ERROR "Please install QPL dependency library:libaccel-config from https://github.com/intel/idxd-config") -endif() - target_link_libraries(_qpl - PRIVATE ${LIBACCEL} + PRIVATE ch_contrib::LIBACCEL PRIVATE ${CMAKE_DL_LIBS}) add_library (ch_contrib::qpl ALIAS _qpl) From 3bc0c13ea049943b4b2f6e21fbe36945101c45fd Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 31 Jan 2023 09:31:26 -0500 Subject: [PATCH 19/78] Update QPL version from v0.3.0 to v1.0.0 --- contrib/qpl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/qpl b/contrib/qpl index becb7a1b15b..d75a29d95d8 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit becb7a1b15bdb4845ec3721a550707ffa51d029d +Subproject commit d75a29d95d8a548297fce3549d21020005364dc8 From 2f5842a7494aa4a84224aa32ae7e0937ad5e8f97 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 31 Jan 2023 20:25:08 +0100 Subject: [PATCH 20/78] Update tests --- tests/integration/test_storage_rabbitmq/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 0430399f5d9..3ccca75a9cd 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1073,7 +1073,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): raise threads = [] - threads_num = 5 + threads_num = 3 for _ in range(threads_num): threads.append(threading.Thread(target=insert)) for thread in threads: @@ -2981,6 +2981,7 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): channel.stop_consuming() consumer.basic_consume(onReceived, queue_name) + consumer.start_consuming() consumer_connection.close() @@ -3006,6 +3007,7 @@ def test_max_rows_per_message(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'custom1', rabbitmq_max_rows_per_message = 3, + rabbitmq_flush_interval_ms = 1000, format_custom_result_before_delimiter = '\n', format_custom_result_after_delimiter = '\n'; @@ -3324,6 +3326,8 @@ def test_rabbitmq_flush_by_block_size(rabbitmq_cluster): CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.rabbitmq; + + SYSTEM STOP MERGES; """ ) From 96b140cc95e35aabc42fdabe17a0b8a9a57d912d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 29 Jan 2023 21:03:38 +0100 Subject: [PATCH 21/78] Rename columns in system.backups: num_files, num_processed_files, processed_files_size num_processed_files -> num_files (BACKUP) / files_read (RESTORE) processed_files_size -> total_size (BACKUP) / bytes_read (RESTORE) --- src/Backups/BackupImpl.cpp | 108 ++++--- src/Backups/BackupImpl.h | 25 +- src/Backups/BackupsWorker.cpp | 28 +- src/Backups/BackupsWorker.h | 26 +- src/Backups/IBackup.h | 31 +- src/Storages/System/StorageSystemBackups.cpp | 36 ++- .../test_backup_restore_new/test.py | 279 ++++++++++-------- .../_gen/cluster_for_concurrency_test.xml | 48 +++ 8 files changed, 356 insertions(+), 225 deletions(-) create mode 100644 tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index e657c1a92c7..3e937bf7f75 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -271,16 +271,22 @@ size_t BackupImpl::getNumFiles() const return num_files; } -size_t BackupImpl::getNumProcessedFiles() const +UInt64 BackupImpl::getTotalSize() const { std::lock_guard lock{mutex}; - return num_processed_files; + return total_size; } -UInt64 BackupImpl::getProcessedFilesSize() const +size_t BackupImpl::getNumEntries() const { std::lock_guard lock{mutex}; - return processed_files_size; + return num_entries; +} + +UInt64 BackupImpl::getSizeOfEntries() const +{ + std::lock_guard lock{mutex}; + return size_of_entries; } UInt64 BackupImpl::getUncompressedSize() const @@ -295,6 +301,18 @@ UInt64 BackupImpl::getCompressedSize() const return compressed_size; } +size_t BackupImpl::getNumReadFiles() const +{ + std::lock_guard lock{mutex}; + return num_read_files; +} + +UInt64 BackupImpl::getNumReadBytes() const +{ + std::lock_guard lock{mutex}; + return num_read_bytes; +} + void BackupImpl::writeBackupMetadata() { assert(!is_internal_backup); @@ -323,12 +341,18 @@ void BackupImpl::writeBackupMetadata() } } - size_t index = 0; - for (const auto & info : all_file_infos) + num_files = all_file_infos.size(); + total_size = 0; + num_entries = 0; + size_of_entries = 0; + + for (size_t i = 0; i != all_file_infos.size(); ++i) { - String prefix = index ? "contents.file[" + std::to_string(index) + "]." : "contents.file."; + const auto & info = all_file_infos[i]; + String prefix = i ? "contents.file[" + std::to_string(i) + "]." : "contents.file."; config->setString(prefix + "name", info.file_name); config->setUInt64(prefix + "size", info.size); + if (info.size) { config->setString(prefix + "checksum", hexChecksum(info.checksum)); @@ -348,8 +372,14 @@ void BackupImpl::writeBackupMetadata() if (info.pos_in_archive != static_cast(-1)) config->setUInt64(prefix + "pos_in_archive", info.pos_in_archive); } - increaseUncompressedSize(info); - ++index; + + total_size += info.size; + bool has_entry = !deduplicate_files || (info.size && (info.size != info.base_size) && (info.data_file_name.empty() || (info.data_file_name == info.file_name))); + if (has_entry) + { + ++num_entries; + size_of_entries += info.size - info.base_size; + } } std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -366,8 +396,7 @@ void BackupImpl::writeBackupMetadata() out->write(str.data(), str.size()); out->finalize(); - increaseUncompressedSize(str.size()); - increaseProcessedSize(str.size()); + uncompressed_size = size_of_entries + str.size(); } @@ -392,8 +421,6 @@ void BackupImpl::readBackupMetadata() String str; readStringUntilEOF(str, *in); - increaseUncompressedSize(str.size()); - increaseProcessedSize(str.size()); Poco::XML::DOMParser dom_parser; Poco::AutoPtr config = dom_parser.parseMemory(str.data(), str.size()); const Poco::XML::Node * config_root = getRootNode(config); @@ -412,6 +439,11 @@ void BackupImpl::readBackupMetadata() if (config_root->getNodeByPath("base_backup_uuid")) base_backup_uuid = parse(getString(config_root, "base_backup_uuid")); + num_files = 0; + total_size = 0; + num_entries = 0; + size_of_entries = 0; + const auto * contents = config_root->getNodeByPath("contents"); for (const Poco::XML::Node * child = contents->firstChild(); child; child = child->nextSibling()) { @@ -456,10 +488,20 @@ void BackupImpl::readBackupMetadata() } coordination->addFileInfo(info); - increaseUncompressedSize(info); + + ++num_files; + total_size += info.size; + bool has_entry = !deduplicate_files || (info.size && (info.size != info.base_size) && (info.data_file_name.empty() || (info.data_file_name == info.file_name))); + if (has_entry) + { + ++num_entries; + size_of_entries += info.size - info.base_size; + } } } + uncompressed_size = size_of_entries + str.size(); + compressed_size = uncompressed_size; if (!use_archives) setCompressedSize(); } @@ -612,7 +654,8 @@ BackupEntryPtr BackupImpl::readFile(const SizeAndChecksum & size_and_checksum) c if (open_mode != OpenMode::READ) throw Exception("Backup is not opened for reading", ErrorCodes::LOGICAL_ERROR); - increaseProcessedSize(size_and_checksum.first); + ++num_read_files; + num_read_bytes += size_and_checksum.first; if (!size_and_checksum.first) { @@ -779,7 +822,8 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { std::lock_guard lock{mutex}; - increaseProcessedSize(info); + ++num_files; + total_size += info.size; } /// Empty file, nothing to backup @@ -908,7 +952,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { LOG_TRACE(log, "Will copy file {}", adjusted_path); - if (!num_files_written) + if (!num_entries) checkLockFile(true); if (use_archives) @@ -950,7 +994,12 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) } } - ++num_files_written; + { + std::lock_guard lock{mutex}; + ++num_entries; + size_of_entries += info.size - info.base_size; + uncompressed_size += info.size - info.base_size; + } } @@ -980,29 +1029,6 @@ void BackupImpl::finalizeWriting() } -void BackupImpl::increaseUncompressedSize(UInt64 file_size) -{ - uncompressed_size += file_size; - ++num_files; -} - -void BackupImpl::increaseUncompressedSize(const FileInfo & info) -{ - if ((info.size > info.base_size) && (info.data_file_name.empty() || (info.data_file_name == info.file_name))) - increaseUncompressedSize(info.size - info.base_size); -} - -void BackupImpl::increaseProcessedSize(UInt64 file_size) const -{ - processed_files_size += file_size; - ++num_processed_files; -} - -void BackupImpl::increaseProcessedSize(const FileInfo & info) -{ - increaseProcessedSize(info.size); -} - void BackupImpl::setCompressedSize() { if (use_archives) diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 45c471aa825..4aa300d5021 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -59,10 +59,13 @@ public: time_t getTimestamp() const override { return timestamp; } UUID getUUID() const override { return *uuid; } size_t getNumFiles() const override; - size_t getNumProcessedFiles() const override; - UInt64 getProcessedFilesSize() const override; + UInt64 getTotalSize() const override; + size_t getNumEntries() const override; + UInt64 getSizeOfEntries() const override; UInt64 getUncompressedSize() const override; UInt64 getCompressedSize() const override; + size_t getNumReadFiles() const override; + UInt64 getNumReadBytes() const override; Strings listFiles(const String & directory, bool recursive) const override; bool hasFiles(const String & directory) const override; bool fileExists(const String & file_name) const override; @@ -103,16 +106,6 @@ private: std::shared_ptr getArchiveReader(const String & suffix) const; std::shared_ptr getArchiveWriter(const String & suffix); - /// Increases `uncompressed_size` by a specific value, - /// also increases `num_files` by 1. - void increaseUncompressedSize(UInt64 file_size); - void increaseUncompressedSize(const FileInfo & info); - - /// Increases `num_processed_files` by a specific value, - /// also increases `num_processed_files` by 1. - void increaseProcessedSize(UInt64 file_size) const; - void increaseProcessedSize(const FileInfo & info); - /// Calculates and sets `compressed_size`. void setCompressedSize(); @@ -129,10 +122,13 @@ private: std::optional uuid; time_t timestamp = 0; size_t num_files = 0; - mutable size_t num_processed_files = 0; - mutable UInt64 processed_files_size = 0; + UInt64 total_size = 0; + size_t num_entries = 0; + UInt64 size_of_entries = 0; UInt64 uncompressed_size = 0; UInt64 compressed_size = 0; + mutable size_t num_read_files = 0; + mutable UInt64 num_read_bytes = 0; int version; std::optional base_backup_info; std::shared_ptr base_backup; @@ -141,7 +137,6 @@ private: std::pair> archive_writers[2]; String current_archive_suffix; String lock_file_name; - std::atomic num_files_written = 0; bool writing_finalized = false; bool deduplicate_files = true; const Poco::Logger * log; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 2c012464ca6..1b0288a44db 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -341,20 +341,20 @@ void BackupsWorker::doBackup( } size_t num_files = 0; - size_t num_processed_files = 0; + UInt64 total_size = 0; + size_t num_entries = 0; UInt64 uncompressed_size = 0; UInt64 compressed_size = 0; - UInt64 processed_files_size = 0; /// Finalize backup (write its metadata). if (!backup_settings.internal) { backup->finalizeWriting(); num_files = backup->getNumFiles(); - num_processed_files = backup->getNumProcessedFiles(); + total_size = backup->getTotalSize(); + num_entries = backup->getNumEntries(); uncompressed_size = backup->getUncompressedSize(); compressed_size = backup->getCompressedSize(); - processed_files_size = backup->getProcessedFilesSize(); } /// Close the backup. @@ -362,7 +362,7 @@ void BackupsWorker::doBackup( LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_name_for_logging); setStatus(backup_id, BackupStatus::BACKUP_CREATED); - setNumFilesAndSize(backup_id, num_files, num_processed_files, processed_files_size, uncompressed_size, compressed_size); + setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0); } catch (...) { @@ -588,10 +588,12 @@ void BackupsWorker::doRestore( setNumFilesAndSize( restore_id, backup->getNumFiles(), - backup->getNumProcessedFiles(), - backup->getProcessedFilesSize(), + backup->getTotalSize(), + backup->getNumEntries(), backup->getUncompressedSize(), - backup->getCompressedSize()); + backup->getCompressedSize(), + backup->getNumReadFiles(), + backup->getNumReadBytes()); } catch (...) { @@ -672,7 +674,9 @@ void BackupsWorker::setStatus(const String & id, BackupStatus status, bool throw } -void BackupsWorker::setNumFilesAndSize(const String & id, size_t num_files, size_t num_processed_files, UInt64 processed_files_size, UInt64 uncompressed_size, UInt64 compressed_size) +void BackupsWorker::setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, + UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes) + { std::lock_guard lock{infos_mutex}; auto it = infos.find(id); @@ -681,10 +685,12 @@ void BackupsWorker::setNumFilesAndSize(const String & id, size_t num_files, size auto & info = it->second; info.num_files = num_files; - info.num_processed_files = num_processed_files; - info.processed_files_size = processed_files_size; + info.total_size = total_size; + info.num_entries = num_entries; info.uncompressed_size = uncompressed_size; info.compressed_size = compressed_size; + info.num_read_files = num_read_files; + info.num_read_bytes = num_read_bytes; } diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 19d2e4d0d1d..40625dc147b 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -53,23 +53,27 @@ public: /// Status of backup or restore operation. BackupStatus status; - /// Number of files in the backup (including backup's metadata; only unique files are counted). + /// The number of files stored in the backup. size_t num_files = 0; - /// Number of processed files during backup or restore process - /// For restore it includes files from base backups - size_t num_processed_files = 0; + /// The total size of files stored in the backup. + UInt64 total_size = 0; - /// Size of processed files during backup or restore - /// For restore in includes sizes from base backups - UInt64 processed_files_size = 0; + /// The number of entries in the backup, i.e. the number of files inside the folder if the backup is stored as a folder. + size_t num_entries = 0; - /// Size of all files in the backup (including backup's metadata; only unique files are counted). + /// The uncompressed size of the backup. UInt64 uncompressed_size = 0; - /// Size of the backup if it's stored as an archive; or the same as `uncompressed_size` if the backup is stored as a folder. + /// The compressed size of the backup. UInt64 compressed_size = 0; + /// Returns the number of files read during RESTORE from this backup. + size_t num_read_files = 0; + + // Returns the total size of files read during RESTORE from this backup. + UInt64 num_read_bytes = 0; + /// Set only if there was an error. std::exception_ptr exception; String error_message; @@ -110,7 +114,9 @@ private: void addInfo(const OperationID & id, const String & name, bool internal, BackupStatus status); void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true); void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } - void setNumFilesAndSize(const OperationID & id, size_t num_files, size_t num_processed_files, UInt64 processed_files_size, UInt64 uncompressed_size, UInt64 compressed_size); + void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, + UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes); + std::vector getAllActiveBackupInfos() const; std::vector getAllActiveRestoreInfos() const; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 13c21fb0287..208305e3f35 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -37,21 +37,38 @@ public: /// Returns UUID of the backup. virtual UUID getUUID() const = 0; - /// Returns the number of unique files in the backup. + /// Returns the number of files stored in the backup. Compare with getNumEntries(). virtual size_t getNumFiles() const = 0; - /// Returns the number of files were processed for backup or restore - virtual size_t getNumProcessedFiles() const = 0; + /// Returns the total size of files stored in the backup. Compare with getTotalSizeOfEntries(). + virtual UInt64 getTotalSize() const = 0; - // Returns the total size of processed files for backup or restore - virtual UInt64 getProcessedFilesSize() const = 0; + /// Returns the number of entries in the backup, i.e. the number of files inside the folder if the backup is stored as a folder or + /// the number of files inside the archive if the backup is stored as an archive. + /// It's not the same as getNumFiles() if it's an incremental backups or if it contains empty files or duplicates. + /// The following is always true: `getNumEntries() <= getNumFiles()`. + virtual size_t getNumEntries() const = 0; - /// Returns the total size of unique files in the backup. + /// Returns the size of entries in the backup, i.e. the total size of files inside the folder if the backup is stored as a folder or + /// the total size of files inside the archive if the backup is stored as an archive. + /// It's not the same as getTotalSize() because it doesn't include the size of duplicates and the size of files from the base backup. + /// The following is always true: `getSizeOfEntries() <= getTotalSize()`. + virtual UInt64 getSizeOfEntries() const = 0; + + /// Returns the uncompressed size of the backup. It equals to `getSizeOfEntries() + size_of_backup_metadata (.backup)` virtual UInt64 getUncompressedSize() const = 0; - /// Returns the compressed size of the backup. If the backup is not stored as an archive it returns the same as getUncompressedSize(). + /// Returns the compressed size of the backup. If the backup is not stored as an archive it's the same as getUncompressedSize(). virtual UInt64 getCompressedSize() const = 0; + /// Returns the number of files read during RESTORE from this backup. + /// The following is always true: `getNumFilesRead() <= getNumFiles()`. + virtual size_t getNumReadFiles() const = 0; + + // Returns the total size of files read during RESTORE from this backup. + /// The following is always true: `getNumReadBytes() <= getTotalSize()`. + virtual UInt64 getNumReadBytes() const = 0; + /// Returns names of entries stored in a specified directory in the backup. /// If `directory` is empty or '/' the functions returns entries in the backup's root. virtual Strings listFiles(const String & directory, bool recursive = false) const = 0; diff --git a/src/Storages/System/StorageSystemBackups.cpp b/src/Storages/System/StorageSystemBackups.cpp index 268cc9d0963..1f448bcbdbc 100644 --- a/src/Storages/System/StorageSystemBackups.cpp +++ b/src/Storages/System/StorageSystemBackups.cpp @@ -18,14 +18,16 @@ NamesAndTypesList StorageSystemBackups::getNamesAndTypes() {"id", std::make_shared()}, {"name", std::make_shared()}, {"status", std::make_shared(getBackupStatusEnumValues())}, - {"num_files", std::make_shared()}, - {"num_processed_files", std::make_shared()}, - {"processed_files_size", std::make_shared()}, - {"uncompressed_size", std::make_shared()}, - {"compressed_size", std::make_shared()}, {"error", std::make_shared()}, {"start_time", std::make_shared()}, {"end_time", std::make_shared()}, + {"num_files", std::make_shared()}, + {"total_size", std::make_shared()}, + {"num_entries", std::make_shared()}, + {"uncompressed_size", std::make_shared()}, + {"compressed_size", std::make_shared()}, + {"files_read", std::make_shared()}, + {"bytes_read", std::make_shared()}, }; return names_and_types; } @@ -37,28 +39,32 @@ void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr con auto & column_id = assert_cast(*res_columns[column_index++]); auto & column_name = assert_cast(*res_columns[column_index++]); auto & column_status = assert_cast(*res_columns[column_index++]); - auto & column_num_files = assert_cast(*res_columns[column_index++]); - auto & column_num_processed_files = assert_cast(*res_columns[column_index++]); - auto & column_processed_files_size = assert_cast(*res_columns[column_index++]); - auto & column_uncompressed_size = assert_cast(*res_columns[column_index++]); - auto & column_compressed_size = assert_cast(*res_columns[column_index++]); auto & column_error = assert_cast(*res_columns[column_index++]); auto & column_start_time = assert_cast(*res_columns[column_index++]); auto & column_end_time = assert_cast(*res_columns[column_index++]); + auto & column_num_files = assert_cast(*res_columns[column_index++]); + auto & column_total_size = assert_cast(*res_columns[column_index++]); + auto & column_num_entries = assert_cast(*res_columns[column_index++]); + auto & column_uncompressed_size = assert_cast(*res_columns[column_index++]); + auto & column_compressed_size = assert_cast(*res_columns[column_index++]); + auto & column_num_read_files = assert_cast(*res_columns[column_index++]); + auto & column_num_read_bytes = assert_cast(*res_columns[column_index++]); auto add_row = [&](const BackupsWorker::Info & info) { column_id.insertData(info.id.data(), info.id.size()); column_name.insertData(info.name.data(), info.name.size()); column_status.insertValue(static_cast(info.status)); - column_num_files.insertValue(info.num_files); - column_num_processed_files.insertValue(info.num_processed_files); - column_processed_files_size.insertValue(info.processed_files_size); - column_uncompressed_size.insertValue(info.uncompressed_size); - column_compressed_size.insertValue(info.compressed_size); column_error.insertData(info.error_message.data(), info.error_message.size()); column_start_time.insertValue(static_cast(std::chrono::system_clock::to_time_t(info.start_time))); column_end_time.insertValue(static_cast(std::chrono::system_clock::to_time_t(info.end_time))); + column_num_files.insertValue(info.num_files); + column_total_size.insertValue(info.total_size); + column_num_entries.insertValue(info.num_entries); + column_uncompressed_size.insertValue(info.uncompressed_size); + column_compressed_size.insertValue(info.compressed_size); + column_num_read_files.insertValue(info.num_read_files); + column_num_read_bytes.insertValue(info.num_read_bytes); }; for (const auto & entry : context->getBackupsWorker().getAllInfos()) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index b1fab60f33b..918998b960a 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,11 +1,14 @@ import pytest import asyncio +import glob import re import random import os.path +from collections import namedtuple from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry, TSV + cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", @@ -62,6 +65,13 @@ def get_path_to_backup(backup_name): return os.path.join(instance.cluster.instances_dir, "backups", name) +def find_files_in_backup_folder(backup_name): + path = get_path_to_backup(backup_name) + files = [f for f in glob.glob(path + "/**", recursive=True) if os.path.isfile(f)] + files += [f for f in glob.glob(path + "/.**", recursive=True) if os.path.isfile(f)] + return files + + session_id_counter = 0 @@ -80,6 +90,63 @@ def has_mutation_in_backup(mutation_id, backup_name, database, table): ) +BackupInfo = namedtuple( + "BackupInfo", + "name id status error num_files total_size num_entries uncompressed_size compressed_size files_read bytes_read", +) + + +def get_backup_info_from_system_backups(by_id=None, by_name=None): + where_condition = "1" + if by_id: + where_condition = f"id = '{by_id}'" + elif by_name: + where_condition = f"name = '{by_name}'" + + [ + name, + id, + status, + error, + num_files, + total_size, + num_entries, + uncompressed_size, + compressed_size, + files_read, + bytes_read, + ] = ( + instance.query( + f"SELECT name, id, status, error, num_files, total_size, num_entries, uncompressed_size, compressed_size, files_read, bytes_read " + f"FROM system.backups WHERE {where_condition} LIMIT 1" + ) + .strip("\n") + .split("\t") + ) + + num_files = int(num_files) + total_size = int(total_size) + num_entries = int(num_entries) + uncompressed_size = int(uncompressed_size) + compressed_size = int(compressed_size) + files_read = int(files_read) + bytes_read = int(bytes_read) + + return BackupInfo( + name=name, + id=id, + status=status, + error=error, + num_files=num_files, + total_size=total_size, + num_entries=num_entries, + uncompressed_size=uncompressed_size, + compressed_size=compressed_size, + files_read=files_read, + bytes_read=bytes_read, + ) + + @pytest.mark.parametrize( "engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"] ) @@ -195,96 +262,62 @@ def test_incremental_backup(): def test_increment_backup_without_changes(): backup_name = new_backup_name() incremental_backup_name = new_backup_name() + create_and_fill_table(n=1) - - system_backup_qry = "SELECT status, num_files, num_processed_files, processed_files_size, uncompressed_size, compressed_size, error FROM system.backups WHERE id='{id_backup}'" - assert instance.query("SELECT count(), sum(x) FROM test.table") == TSV([["1", "0"]]) # prepare first backup without base_backup - (id_backup, status) = instance.query( - f"BACKUP TABLE test.table TO {backup_name}" - ).split("\t") + id_backup = instance.query(f"BACKUP TABLE test.table TO {backup_name}").split("\t")[ + 0 + ] + backup_info = get_backup_info_from_system_backups(by_id=id_backup) - ( - backup_status, - num_files, - num_processed_files, - processed_files_size, - uncompressed_size, - compressed_size, - error, - ) = ( - instance.query(system_backup_qry.format(id_backup=id_backup)) - .strip("\n") - .split("\t") + assert backup_info.status == "BACKUP_CREATED" + assert backup_info.error == "" + assert backup_info.num_files > 0 + assert backup_info.total_size > 0 + assert ( + 0 < backup_info.num_entries and backup_info.num_entries <= backup_info.num_files ) - - assert backup_status == "BACKUP_CREATED" - assert num_files == "11" - assert int(uncompressed_size) > 0 - assert int(compressed_size) > 0 - assert error == "" + assert backup_info.uncompressed_size > 0 + assert backup_info.compressed_size == backup_info.uncompressed_size # create second backup without changes based on the first one - (id_backup_wo_changes, status_backup_wo_changes) = instance.query( + id_backup2 = instance.query( f"BACKUP TABLE test.table TO {incremental_backup_name} SETTINGS base_backup = {backup_name}" - ).split("\t") + ).split("\t")[0] - ( - backup_status_wo_changes, - num_files_backup_wo_changes, - num_processed_files_backup_wo_changes, - processed_files_size_backup_wo_changes, - uncompressed_size_backup_wo_changes, - compressed_size_backup_wo_changes, - error_snd, - ) = ( - instance.query(system_backup_qry.format(id_backup=id_backup_wo_changes)) - .strip("\n") - .split("\t") - ) + backup2_info = get_backup_info_from_system_backups(by_id=id_backup2) - assert backup_status_wo_changes == "BACKUP_CREATED" - assert num_files_backup_wo_changes == "1" - assert num_processed_files_backup_wo_changes == "11" - assert int(processed_files_size_backup_wo_changes) > 0 - assert int(uncompressed_size_backup_wo_changes) > 0 - assert int(compressed_size_backup_wo_changes) > 0 - assert error_snd == "" + assert backup2_info.status == "BACKUP_CREATED" + assert backup2_info.error == "" + assert backup2_info.num_files == backup_info.num_files + assert backup2_info.total_size == backup_info.total_size + assert backup2_info.num_entries == 0 + assert backup2_info.uncompressed_size > 0 + assert backup2_info.compressed_size == backup2_info.uncompressed_size # restore the second backup # we expect to see all files in the meta info of the restore and a sum of uncompressed and compressed sizes - (id_restore, status_restore) = instance.query( + id_restore = instance.query( f"RESTORE TABLE test.table AS test.table2 FROM {incremental_backup_name}" - ).split("\t") + ).split("\t")[0] assert instance.query("SELECT count(), sum(x) FROM test.table2") == TSV( [["1", "0"]] ) - ( - restore_status, - restore_num_files, - restore_num_processed_files, - restore_processed_files_size, - restore_uncompressed_size, - restore_compressed_size, - restore_error, - ) = ( - instance.query(system_backup_qry.format(id_backup=id_restore)) - .strip("\n") - .split("\t") - ) + restore_info = get_backup_info_from_system_backups(by_id=id_restore) - assert restore_status == "RESTORED" - assert int(restore_num_files) == 1 - assert int(restore_num_processed_files) == int( - num_processed_files_backup_wo_changes - ) - assert int(restore_uncompressed_size) > 0 - assert int(restore_compressed_size) > 0 - assert restore_error == "" + assert restore_info.status == "RESTORED" + assert restore_info.error == "" + assert restore_info.num_files == backup2_info.num_files + assert restore_info.total_size == backup2_info.total_size + assert restore_info.num_entries == backup2_info.num_entries + assert restore_info.uncompressed_size == backup2_info.uncompressed_size + assert restore_info.compressed_size == backup2_info.compressed_size + assert restore_info.files_read == backup2_info.num_files + assert restore_info.bytes_read == backup2_info.total_size def test_incremental_backup_overflow(): @@ -1178,44 +1211,55 @@ def test_operation_id(): def test_system_backups(): + # Backup create_and_fill_table(n=30) backup_name = new_backup_name() - id = instance.query(f"BACKUP TABLE test.table TO {backup_name}").split("\t")[0] - [ - name, - status, - num_files, - num_processed_files, - processed_files_size, - uncompressed_size, - compressed_size, - error, - ] = ( - instance.query( - f"SELECT name, status, num_files, num_processed_files, processed_files_size, uncompressed_size, compressed_size, error FROM system.backups WHERE id='{id}'" - ) - .strip("\n") - .split("\t") + info = get_backup_info_from_system_backups(by_id=id) + escaped_backup_name = backup_name.replace("'", "\\'") + assert info.name == escaped_backup_name + assert info.status == "BACKUP_CREATED" + assert info.error == "" + assert info.num_files > 0 + assert info.total_size > 0 + assert 0 < info.num_entries and info.num_entries <= info.num_files + assert info.uncompressed_size > 0 + assert info.compressed_size == info.uncompressed_size + assert info.files_read == 0 + assert info.bytes_read == 0 + + files_in_backup_folder = find_files_in_backup_folder(backup_name) + assert info.num_entries == len(files_in_backup_folder) - 1 + assert info.uncompressed_size == sum( + os.path.getsize(f) for f in files_in_backup_folder ) - escaped_backup_name = backup_name.replace("'", "\\'") - num_files = int(num_files) - compressed_size = int(compressed_size) - uncompressed_size = int(uncompressed_size) - num_processed_files = int(num_processed_files) - processed_files_size = int(processed_files_size) - assert name == escaped_backup_name - assert status == "BACKUP_CREATED" - assert num_files > 1 - assert num_processed_files > 1 - assert processed_files_size > 1 - assert uncompressed_size > 1 - assert compressed_size == uncompressed_size - assert error == "" + # The concrete values can change. + info.num_files == 91 + info.total_size == 4973 + info.num_entries == 55 + info.uncompressed_size == 19701 + instance.query("DROP TABLE test.table") + + # Restore + id = instance.query(f"RESTORE TABLE test.table FROM {backup_name}").split("\t")[0] + restore_info = get_backup_info_from_system_backups(by_id=id) + + assert restore_info.name == escaped_backup_name + assert restore_info.status == "RESTORED" + assert restore_info.error == "" + assert restore_info.num_files == info.num_files + assert restore_info.total_size == info.total_size + assert restore_info.num_entries == info.num_entries + assert restore_info.uncompressed_size == info.uncompressed_size + assert restore_info.compressed_size == info.compressed_size + assert restore_info.files_read == restore_info.num_files + assert restore_info.bytes_read == restore_info.total_size + + # Failed backup. backup_name = new_backup_name() expected_error = "Table test.non_existent_table was not found" assert expected_error in instance.query_and_get_error( @@ -1223,34 +1267,17 @@ def test_system_backups(): ) escaped_backup_name = backup_name.replace("'", "\\'") - [ - status, - num_files, - num_processed_files, - processed_files_size, - uncompressed_size, - compressed_size, - error, - ] = ( - instance.query( - f"SELECT status, num_files, num_processed_files, processed_files_size, uncompressed_size, compressed_size, error FROM system.backups WHERE name='{escaped_backup_name}'" - ) - .strip("\n") - .split("\t") - ) + info = get_backup_info_from_system_backups(by_name=escaped_backup_name) - num_files = int(num_files) - compressed_size = int(compressed_size) - uncompressed_size = int(uncompressed_size) - num_processed_files = int(num_processed_files) - processed_files_size = int(processed_files_size) - assert status == "BACKUP_FAILED" - assert num_files == 0 - assert uncompressed_size == 0 - assert compressed_size == 0 - assert num_processed_files == 0 - assert processed_files_size == 0 - assert expected_error in error + assert info.status == "BACKUP_FAILED" + assert expected_error in info.error + assert info.num_files == 0 + assert info.total_size == 0 + assert info.num_entries == 0 + assert info.uncompressed_size == 0 + assert info.compressed_size == 0 + assert info.files_read == 0 + assert info.bytes_read == 0 def test_mutation(): diff --git a/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml b/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml new file mode 100644 index 00000000000..08684e34e45 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/_gen/cluster_for_concurrency_test.xml @@ -0,0 +1,48 @@ + + + + + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + node4 + 9000 + + + node5 + 9000 + + + node6 + 9000 + + + node7 + 9000 + + + node8 + 9000 + + + node9 + 9000 + + + + + \ No newline at end of file From 5c4db119fa25dbc5704bb360424ef120530397aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 1 Feb 2023 03:11:54 +0100 Subject: [PATCH 22/78] Fix trash around query kind, part 1 --- src/Interpreters/Access/InterpreterGrantQuery.cpp | 6 +++--- src/Interpreters/Access/InterpreterGrantQuery.h | 3 ++- src/Interpreters/IInterpreter.cpp | 7 +++++++ src/Interpreters/IInterpreter.h | 3 +++ src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 5 ++--- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 6 +++++- src/Interpreters/InterpreterAlterQuery.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 6 +++++- src/Interpreters/InterpreterCreateQuery.h | 1 + src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 5 ++++- src/Interpreters/InterpreterInsertQuery.h | 2 ++ src/Interpreters/InterpreterRenameQuery.cpp | 6 +++++- src/Interpreters/InterpreterRenameQuery.h | 2 ++ .../InterpreterSelectIntersectExceptQuery.cpp | 7 +++++-- .../InterpreterSelectIntersectExceptQuery.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 7 +++++-- src/Interpreters/InterpreterSelectQuery.h | 1 + src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 4 ++-- src/Interpreters/InterpreterSelectQueryAnalyzer.h | 2 +- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 7 +++++-- src/Interpreters/InterpreterSelectWithUnionQuery.h | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 4 ++-- src/Interpreters/InterpreterSystemQuery.h | 2 +- src/Interpreters/ProcessList.cpp | 11 ++++------- src/Interpreters/ProcessList.h | 10 ++++++---- src/Parsers/ASTSelectIntersectExceptQuery.h | 2 +- src/Parsers/IAST.h | 1 - src/Storages/System/StorageSystemProcesses.cpp | 2 ++ 31 files changed, 83 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 7f2093a22a7..2a432d748be 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -419,13 +419,13 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery } -void InterpreterGrantQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr) const +String InterpreterGrantQuery::getQueryKind() const { auto & query = query_ptr->as(); if (query.is_revoke) - elem.query_kind = "Revoke"; + return "Revoke"; else - elem.query_kind = "Grant"; + return "Grant"; } } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.h b/src/Interpreters/Access/InterpreterGrantQuery.h index abaddcc599b..cb0e6e403d2 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.h +++ b/src/Interpreters/Access/InterpreterGrantQuery.h @@ -21,7 +21,8 @@ public: static void updateUserFromQuery(User & user, const ASTGrantQuery & query); static void updateRoleFromQuery(Role & role, const ASTGrantQuery & query); - void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const override; + + String getQueryKind() const override; private: ASTPtr query_ptr; diff --git a/src/Interpreters/IInterpreter.cpp b/src/Interpreters/IInterpreter.cpp index 84fbfee7905..d21707626f1 100644 --- a/src/Interpreters/IInterpreter.cpp +++ b/src/Interpreters/IInterpreter.cpp @@ -11,6 +11,12 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } + +String IInterpreter::getQueryKind() const +{ + return {}; +} + void IInterpreter::extendQueryLogElem( QueryLogElement & elem, const ASTPtr & ast, ContextPtr context, const String & query_database, const String & query_table) const { @@ -26,6 +32,7 @@ void IInterpreter::extendQueryLogElem( elem.query_tables.insert(quoted_database + "." + backQuoteIfNeed(query_table)); } + elem.query_kind = getQueryKind(); extendQueryLogElemImpl(elem, ast, context); } diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 74a568c5cba..8000013ecb0 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -32,6 +32,9 @@ public: const String & query_database, const String & query_table) const; + /// Get a string identifying the type of query (e.g. "Select"). + virtual String getQueryKind() const; + virtual void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const {} /// Returns true if transactions maybe supported for this type of query. diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 5df3abbb32e..7d6f87baa35 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -9,16 +9,15 @@ #include #include #include -#include #include namespace DB { -void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const +String IInterpreterUnionOrSelectQuery::getQueryKind() const { - elem.query_kind = "Select"; + return "Select"; } QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 0addaa7e024..afb9f941d4b 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -44,7 +44,7 @@ public: size_t getMaxStreams() const { return max_streams; } - void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + String getQueryKind() const override; /// Returns whether the query uses the view source from the Context /// The view source is a virtual storage that currently only materialized views use to replace the source table diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index ee5aad3d18e..a43bc9bcbca 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -442,11 +442,15 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS return required_access; } +String InterpreterAlterQuery::getQueryKind() const +{ + return "Alter"; +} + void InterpreterAlterQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const { const auto & alter = ast->as(); - elem.query_kind = "Alter"; if (alter.command_list != nullptr && alter.alter_object != ASTAlterQuery::AlterObjectType::DATABASE) { // Alter queries already have their target table inserted into `elem`. diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index c6648ff9e7e..1493811a35d 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -24,6 +24,7 @@ public: static AccessRightsElements getRequiredAccessForCommand(const ASTAlterCommand & command, const String & database, const String & table); + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; bool supportsTransactions() const override { return true; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 611f533d559..3c2cf851557 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1701,9 +1701,13 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const return required_access; } +String InterpreterCreateQuery::getQueryKind() const +{ + return "Create"; +} + void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const { - elem.query_kind = "Create"; if (!as_table_saved.empty()) { String database = backQuoteIfNeed(as_database_saved.empty() ? getContext()->getCurrentDatabase() : as_database_saved); diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 4d11387f44c..fa4943cc8f8 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -68,6 +68,7 @@ public: static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name); + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; private: diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index f2f937f6ec0..5daa8d745e0 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -432,9 +432,9 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co return required_access; } -void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const +String InterpreterDropQuery::getQueryKind() const { - elem.query_kind = "Drop"; + return "Drop"; } void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync) diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index edd84471c22..8635c8963ad 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -24,7 +24,7 @@ public: /// Drop table or database. BlockIO execute() override; - void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + String getQueryKind() const override; static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 3b90ac8d284..ebf6746b243 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -561,10 +561,13 @@ StorageID InterpreterInsertQuery::getDatabaseTable() const return query_ptr->as().table_id; } +String InterpreterInsertQuery::getQueryKind() const +{ + return "Insert"; +} void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_) { - elem.query_kind = "Insert"; const auto & insert_table = context_->getInsertionTable(); if (!insert_table.empty()) { diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 842460c8c25..824c490e0ad 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -44,6 +44,8 @@ public: std::atomic_uint64_t * elapsed_counter_ms = nullptr); static void extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_); + + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; StoragePtr getTable(ASTInsertQuery & query); diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 82c230ef8e2..d8db56b4f86 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -195,9 +195,13 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename return required_access; } +String InterpreterRenameQuery::getQueryKind() const +{ + return "Rename"; +} + void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const { - elem.query_kind = "Rename"; const auto & rename = ast->as(); for (const auto & element : rename.elements) { diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index 6141e8c1585..b7db08a63b6 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -55,6 +55,8 @@ class InterpreterRenameQuery : public IInterpreter, WithContext public: InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_); BlockIO execute() override; + + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; bool renamedInsteadOfExchange() const { return renamed_instead_of_exchange; } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 15945cbbeef..4c00947db10 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -191,10 +191,13 @@ void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } +String InterpreterSelectIntersectExceptQuery::getQueryKind() const +{ + return "Select"; +} + void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { - elem.query_kind = "Select"; - for (const auto & interpreter : nested_interpreters) { if (const auto * select_interpreter = dynamic_cast(interpreter.get())) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index e93ab7bd2e9..401ce005430 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -35,6 +35,7 @@ public: void ignoreWithTotals() override; + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; private: diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 624859300b9..6647a200494 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1923,10 +1923,13 @@ RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const return row_policy_filter; } +String InterpreterSelectQuery::getQueryKind() const +{ + return "Select"; +} + void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { - elem.query_kind = "Select"; - for (const auto & row_policy : row_policy_filter->policies) { auto name = row_policy->getFullName().toString(); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 761eea8e1b8..cbb83e9885b 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -136,6 +136,7 @@ public: RowPolicyFilterPtr getRowPolicyFilter() const; + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 86417701f6d..fd837b67155 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -135,9 +135,9 @@ void InterpreterSelectQueryAnalyzer::addStorageLimits(const StorageLimitsList & planner.addStorageLimits(storage_limits); } -void InterpreterSelectQueryAnalyzer::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const +String InterpreterSelectQueryAnalyzer::getQueryKind() const { - elem.query_kind = "Select"; + return "Select"; } void InterpreterSelectQueryAnalyzer::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 0c2465224e7..a51574a7866 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -46,7 +46,7 @@ public: bool ignoreQuota() const override { return select_query_options.ignore_quota; } - void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + String getQueryKind() const override; /// Set merge tree read task callback in context and set collaborate_with_initiator in client info void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e3954f2a197..e7fb7c23a2b 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -396,10 +396,13 @@ void InterpreterSelectWithUnionQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } +String InterpreterSelectWithUnionQuery::getQueryKind() const +{ + return "Select"; +} + void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { - elem.query_kind = "Select"; - for (const auto & interpreter : nested_interpreters) { if (const auto * select_interpreter = dynamic_cast(interpreter.get())) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index 269020bf4da..a3c05b010e1 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -48,6 +48,7 @@ public: bool supportsTransactions() const override { return true; } + String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; private: diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 19b31a858f4..6003e4a2a54 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1155,9 +1155,9 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() return required_access; } -void InterpreterSystemQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr) const +String InterpreterSystemQuery::getQueryKind() const { - elem.query_kind = "System"; + return "System"; } } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 5673890daf3..0c3bf744d2d 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -74,7 +74,7 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); - void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const override; + String getQueryKind() const override; }; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 9ee6cf1afe8..0792b5efd3e 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -3,17 +3,13 @@ #include #include #include -#include #include #include #include #include -#include #include -#include #include #include -#include #include #include @@ -291,7 +287,7 @@ ProcessListEntry::~ProcessListEntry() String user = (*it)->getClientInfo().current_user; String query_id = (*it)->getClientInfo().current_query_id; - IAST::QueryKind query_kind = (*it)->query_kind; + IAST::QueryKind query_kind = (*it)->ast_query_kind; const QueryStatusPtr process_list_element_ptr = *it; @@ -347,7 +343,7 @@ QueryStatus::QueryStatus( const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, ThreadGroupStatusPtr && thread_group_, - IAST::QueryKind query_kind_, + IAST::QueryKind ast_query_kind_, UInt64 watch_start_nanoseconds) : WithContext(context_) , query(query_) @@ -356,7 +352,7 @@ QueryStatus::QueryStatus( , watch(CLOCK_MONOTONIC, watch_start_nanoseconds, true) , priority_handle(std::move(priority_handle_)) , global_overcommit_tracker(context_->getGlobalOvercommitTracker()) - , query_kind(query_kind_) + , ast_query_kind(ast_query_kind_) , num_queries_increment(CurrentMetrics::Query) { auto settings = getContext()->getSettings(); @@ -526,6 +522,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even QueryStatusInfo res{}; res.query = query; + res.ast_query_kind = ast_query_kind; res.client_info = client_info; res.elapsed_microseconds = watch.elapsedMicroseconds(); res.is_cancelled = is_killed.load(std::memory_order_relaxed); diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index eae8b15c695..a8af31c35c0 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -52,6 +52,7 @@ class ProcessListEntry; struct QueryStatusInfo { String query; + IAST::QueryKind ast_query_kind; UInt64 elapsed_microseconds; size_t read_rows; size_t read_bytes; @@ -134,7 +135,8 @@ protected: OvercommitTracker * global_overcommit_tracker = nullptr; - IAST::QueryKind query_kind; + /// This is used to control the maximum number of SELECT or INSERT queries. + IAST::QueryKind ast_query_kind; /// This field is unused in this class, but it /// increments/decrements metric in constructor/destructor. @@ -147,7 +149,7 @@ public: const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, ThreadGroupStatusPtr && thread_group_, - IAST::QueryKind query_kind_, + IAST::QueryKind ast_query_kind_, UInt64 watch_start_nanoseconds); ~QueryStatus(); @@ -176,9 +178,9 @@ public: return &thread_group->memory_tracker; } - IAST::QueryKind getQueryKind() const + IAST::QueryKind getASTQueryKind() const { - return query_kind; + return ast_query_kind; } bool updateProgressIn(const Progress & value) diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index ad962fe25e2..db00fb3df87 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -25,7 +25,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - QueryKind getQueryKind() const override { return QueryKind::SelectIntersectExcept; } + QueryKind getQueryKind() const override { return QueryKind::Select; } ASTs getListOfSelects() const; diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c1520a6fca7..460ecca7ea0 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -260,7 +260,6 @@ public: Insert, Rename, Revoke, - SelectIntersectExcept, Select, System, }; diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index 213e3ed5dc0..b633647c1bb 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -61,6 +61,7 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes() {"memory_usage", std::make_shared()}, {"peak_memory_usage", std::make_shared()}, {"query", std::make_shared()}, + {"query_kind", std::make_shared()}, {"thread_ids", std::make_shared(std::make_shared())}, {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, @@ -130,6 +131,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c res_columns[i++]->insert(process.memory_usage); res_columns[i++]->insert(process.peak_memory_usage); res_columns[i++]->insert(process.query); + res_columns[i++]->insert(magic_enum::enum_name(process.ast_query_kind)); { Array threads_array; From 9b908a0be0bcf2b25ea289d50b250e80b31c43df Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 1 Feb 2023 03:31:31 +0100 Subject: [PATCH 23/78] Fix trash around query kind, part 2 --- src/Interpreters/Access/InterpreterGrantQuery.cpp | 10 ---------- src/Interpreters/Access/InterpreterGrantQuery.h | 2 -- src/Interpreters/IInterpreter.cpp | 6 ------ src/Interpreters/IInterpreter.h | 3 --- src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 5 ----- src/Interpreters/IInterpreterUnionOrSelectQuery.h | 2 -- src/Interpreters/InterpreterAlterQuery.cpp | 5 ----- src/Interpreters/InterpreterAlterQuery.h | 1 - src/Interpreters/InterpreterCreateQuery.cpp | 5 ----- src/Interpreters/InterpreterCreateQuery.h | 1 - src/Interpreters/InterpreterDropQuery.cpp | 5 ----- src/Interpreters/InterpreterDropQuery.h | 2 -- src/Interpreters/InterpreterInsertQuery.cpp | 5 ----- src/Interpreters/InterpreterInsertQuery.h | 1 - src/Interpreters/InterpreterRenameQuery.cpp | 5 ----- src/Interpreters/InterpreterRenameQuery.h | 1 - .../InterpreterSelectIntersectExceptQuery.cpp | 5 ----- .../InterpreterSelectIntersectExceptQuery.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 5 ----- src/Interpreters/InterpreterSelectQuery.h | 1 - src/Interpreters/InterpreterSelectQueryAnalyzer.cpp | 5 ----- src/Interpreters/InterpreterSelectQueryAnalyzer.h | 2 -- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 5 ----- src/Interpreters/InterpreterSelectWithUnionQuery.h | 1 - src/Interpreters/InterpreterSystemQuery.cpp | 5 ----- src/Interpreters/InterpreterSystemQuery.h | 2 -- src/Interpreters/ProcessList.cpp | 8 ++++---- src/Interpreters/ProcessList.h | 11 +++-------- src/Interpreters/QueryLog.cpp | 4 +++- src/Interpreters/QueryLog.h | 3 ++- src/Interpreters/executeQuery.cpp | 5 +++-- src/Storages/System/StorageSystemProcesses.cpp | 2 +- 32 files changed, 16 insertions(+), 108 deletions(-) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 2a432d748be..f7e170965e2 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -418,14 +418,4 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery updateFromQuery(role, query); } - -String InterpreterGrantQuery::getQueryKind() const -{ - auto & query = query_ptr->as(); - if (query.is_revoke) - return "Revoke"; - else - return "Grant"; -} - } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.h b/src/Interpreters/Access/InterpreterGrantQuery.h index cb0e6e403d2..2e36ac4dcea 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.h +++ b/src/Interpreters/Access/InterpreterGrantQuery.h @@ -22,8 +22,6 @@ public: static void updateUserFromQuery(User & user, const ASTGrantQuery & query); static void updateRoleFromQuery(Role & role, const ASTGrantQuery & query); - String getQueryKind() const override; - private: ASTPtr query_ptr; }; diff --git a/src/Interpreters/IInterpreter.cpp b/src/Interpreters/IInterpreter.cpp index d21707626f1..aff703f79af 100644 --- a/src/Interpreters/IInterpreter.cpp +++ b/src/Interpreters/IInterpreter.cpp @@ -12,11 +12,6 @@ namespace ErrorCodes } -String IInterpreter::getQueryKind() const -{ - return {}; -} - void IInterpreter::extendQueryLogElem( QueryLogElement & elem, const ASTPtr & ast, ContextPtr context, const String & query_database, const String & query_table) const { @@ -32,7 +27,6 @@ void IInterpreter::extendQueryLogElem( elem.query_tables.insert(quoted_database + "." + backQuoteIfNeed(query_table)); } - elem.query_kind = getQueryKind(); extendQueryLogElemImpl(elem, ast, context); } diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 8000013ecb0..74a568c5cba 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -32,9 +32,6 @@ public: const String & query_database, const String & query_table) const; - /// Get a string identifying the type of query (e.g. "Select"). - virtual String getQueryKind() const; - virtual void extendQueryLogElemImpl(QueryLogElement &, const ASTPtr &, ContextPtr) const {} /// Returns true if transactions maybe supported for this type of query. diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 7d6f87baa35..b6e910eac94 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -15,11 +15,6 @@ namespace DB { -String IInterpreterUnionOrSelectQuery::getQueryKind() const -{ - return "Select"; -} - QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() { QueryPlan query_plan; diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index afb9f941d4b..1147070f48a 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -44,8 +44,6 @@ public: size_t getMaxStreams() const { return max_streams; } - String getQueryKind() const override; - /// Returns whether the query uses the view source from the Context /// The view source is a virtual storage that currently only materialized views use to replace the source table /// with the incoming block only diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index a43bc9bcbca..9369aa1cfdc 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -442,11 +442,6 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS return required_access; } -String InterpreterAlterQuery::getQueryKind() const -{ - return "Alter"; -} - void InterpreterAlterQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const { const auto & alter = ast->as(); diff --git a/src/Interpreters/InterpreterAlterQuery.h b/src/Interpreters/InterpreterAlterQuery.h index 1493811a35d..c6648ff9e7e 100644 --- a/src/Interpreters/InterpreterAlterQuery.h +++ b/src/Interpreters/InterpreterAlterQuery.h @@ -24,7 +24,6 @@ public: static AccessRightsElements getRequiredAccessForCommand(const ASTAlterCommand & command, const String & database, const String & table); - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; bool supportsTransactions() const override { return true; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3c2cf851557..e1a2c5775b7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1701,11 +1701,6 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const return required_access; } -String InterpreterCreateQuery::getQueryKind() const -{ - return "Create"; -} - void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const { if (!as_table_saved.empty()) diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index fa4943cc8f8..4d11387f44c 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -68,7 +68,6 @@ public: static void prepareOnClusterQuery(ASTCreateQuery & create, ContextPtr context, const String & cluster_name); - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; private: diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 5daa8d745e0..e4bf22730b3 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -432,11 +432,6 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co return required_access; } -String InterpreterDropQuery::getQueryKind() const -{ - return "Drop"; -} - void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync) { if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 8635c8963ad..afec26424ba 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -24,8 +24,6 @@ public: /// Drop table or database. BlockIO execute() override; - String getQueryKind() const override; - static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool sync); bool supportsTransactions() const override; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index ebf6746b243..4cc9dbf0909 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -561,11 +561,6 @@ StorageID InterpreterInsertQuery::getDatabaseTable() const return query_ptr->as().table_id; } -String InterpreterInsertQuery::getQueryKind() const -{ - return "Insert"; -} - void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_) { const auto & insert_table = context_->getInsertionTable(); diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 824c490e0ad..9b3f617e4b3 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -45,7 +45,6 @@ public: static void extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_); - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; StoragePtr getTable(ASTInsertQuery & query); diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index d8db56b4f86..8cf39d2c850 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -195,11 +195,6 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename return required_access; } -String InterpreterRenameQuery::getQueryKind() const -{ - return "Rename"; -} - void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const { const auto & rename = ast->as(); diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index b7db08a63b6..31d3d3d6ad9 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -56,7 +56,6 @@ public: InterpreterRenameQuery(const ASTPtr & query_ptr_, ContextPtr context_); BlockIO execute() override; - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; bool renamedInsteadOfExchange() const { return renamed_instead_of_exchange; } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 4c00947db10..2218ed4417b 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -191,11 +191,6 @@ void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -String InterpreterSelectIntersectExceptQuery::getQueryKind() const -{ - return "Select"; -} - void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { for (const auto & interpreter : nested_interpreters) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index 401ce005430..e93ab7bd2e9 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -35,7 +35,6 @@ public: void ignoreWithTotals() override; - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; private: diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6647a200494..8e11da479c0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1923,11 +1923,6 @@ RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const return row_policy_filter; } -String InterpreterSelectQuery::getQueryKind() const -{ - return "Select"; -} - void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { for (const auto & row_policy : row_policy_filter->policies) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index cbb83e9885b..761eea8e1b8 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -136,7 +136,6 @@ public: RowPolicyFilterPtr getRowPolicyFilter() const; - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index fd837b67155..85bc70e9382 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -135,11 +135,6 @@ void InterpreterSelectQueryAnalyzer::addStorageLimits(const StorageLimitsList & planner.addStorageLimits(storage_limits); } -String InterpreterSelectQueryAnalyzer::getQueryKind() const -{ - return "Select"; -} - void InterpreterSelectQueryAnalyzer::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback) { context->getClientInfo().collaborate_with_initiator = true; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index a51574a7866..7fe64a7fb99 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -46,8 +46,6 @@ public: bool ignoreQuota() const override { return select_query_options.ignore_quota; } - String getQueryKind() const override; - /// Set merge tree read task callback in context and set collaborate_with_initiator in client info void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e7fb7c23a2b..bfa3d16bf29 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -396,11 +396,6 @@ void InterpreterSelectWithUnionQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -String InterpreterSelectWithUnionQuery::getQueryKind() const -{ - return "Select"; -} - void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { for (const auto & interpreter : nested_interpreters) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index a3c05b010e1..269020bf4da 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -48,7 +48,6 @@ public: bool supportsTransactions() const override { return true; } - String getQueryKind() const override; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; private: diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 6003e4a2a54..a4939b674ca 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1155,9 +1155,4 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() return required_access; } -String InterpreterSystemQuery::getQueryKind() const -{ - return "System"; -} - } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 0c3bf744d2d..df06a2fa4ef 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -73,8 +73,6 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); - - String getQueryKind() const override; }; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 0792b5efd3e..49d7989ac5e 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -287,7 +287,7 @@ ProcessListEntry::~ProcessListEntry() String user = (*it)->getClientInfo().current_user; String query_id = (*it)->getClientInfo().current_query_id; - IAST::QueryKind query_kind = (*it)->ast_query_kind; + IAST::QueryKind query_kind = (*it)->query_kind; const QueryStatusPtr process_list_element_ptr = *it; @@ -343,7 +343,7 @@ QueryStatus::QueryStatus( const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, ThreadGroupStatusPtr && thread_group_, - IAST::QueryKind ast_query_kind_, + IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds) : WithContext(context_) , query(query_) @@ -352,7 +352,7 @@ QueryStatus::QueryStatus( , watch(CLOCK_MONOTONIC, watch_start_nanoseconds, true) , priority_handle(std::move(priority_handle_)) , global_overcommit_tracker(context_->getGlobalOvercommitTracker()) - , ast_query_kind(ast_query_kind_) + , query_kind(query_kind_) , num_queries_increment(CurrentMetrics::Query) { auto settings = getContext()->getSettings(); @@ -522,7 +522,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even QueryStatusInfo res{}; res.query = query; - res.ast_query_kind = ast_query_kind; + res.query_kind = query_kind; res.client_info = client_info; res.elapsed_microseconds = watch.elapsedMicroseconds(); res.is_cancelled = is_killed.load(std::memory_order_relaxed); diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index a8af31c35c0..6756224addc 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -52,7 +52,7 @@ class ProcessListEntry; struct QueryStatusInfo { String query; - IAST::QueryKind ast_query_kind; + IAST::QueryKind query_kind; UInt64 elapsed_microseconds; size_t read_rows; size_t read_bytes; @@ -136,7 +136,7 @@ protected: OvercommitTracker * global_overcommit_tracker = nullptr; /// This is used to control the maximum number of SELECT or INSERT queries. - IAST::QueryKind ast_query_kind; + IAST::QueryKind query_kind; /// This field is unused in this class, but it /// increments/decrements metric in constructor/destructor. @@ -149,7 +149,7 @@ public: const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, ThreadGroupStatusPtr && thread_group_, - IAST::QueryKind ast_query_kind_, + IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds); ~QueryStatus(); @@ -178,11 +178,6 @@ public: return &thread_group->memory_tracker; } - IAST::QueryKind getASTQueryKind() const - { - return ast_query_kind; - } - bool updateProgressIn(const Progress & value) { CurrentThread::updateProgressIn(value); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 224ca9f11a8..2a1c53e746b 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -166,7 +166,9 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(query.data(), query.size()); columns[i++]->insertData(formatted_query.data(), formatted_query.size()); columns[i++]->insert(normalized_query_hash); - columns[i++]->insertData(query_kind.data(), query_kind.size()); + + const std::string_view query_kind_str = magic_enum::enum_name(query_kind); + columns[i++]->insertData(query_kind_str.data(), query_kind_str.size()); { auto & column_databases = typeid_cast(*columns[i++]); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index e8dbbf685ea..00bdf9df30c 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents @@ -58,7 +59,7 @@ struct QueryLogElement String formatted_query; UInt64 normalized_query_hash{}; - String query_kind; + IAST::QueryKind query_kind; std::set query_databases; std::set query_tables; std::set query_columns; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 646f1e89fc3..25d30761451 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -235,10 +235,10 @@ static void onExceptionBeforeStart( elem.query = query_for_logging; elem.normalized_query_hash = normalizedQueryHash(query_for_logging); - // Try log query_kind if ast is valid + // Log query_kind if ast is valid if (ast) { - elem.query_kind = magic_enum::enum_name(ast->getQueryKind()); + elem.query_kind = ast->getQueryKind(); if (settings.log_formatted_queries) elem.formatted_query = queryToString(ast); } @@ -807,6 +807,7 @@ static std::tuple executeQueryImpl( if (settings.log_formatted_queries) elem.formatted_query = queryToString(ast); elem.normalized_query_hash = normalizedQueryHash(query_for_logging); + elem.query_kind = ast->getQueryKind(); elem.client_info = client_info; diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index b633647c1bb..ecc0f781c06 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -131,7 +131,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c res_columns[i++]->insert(process.memory_usage); res_columns[i++]->insert(process.peak_memory_usage); res_columns[i++]->insert(process.query); - res_columns[i++]->insert(magic_enum::enum_name(process.ast_query_kind)); + res_columns[i++]->insert(magic_enum::enum_name(process.query_kind)); { Array threads_array; From cda9b0beeae443ba6434a891674dfb629fa68426 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 1 Feb 2023 15:54:10 +0800 Subject: [PATCH 24/78] optimize parquet reader --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 50 ++++++++++--------- .../Formats/Impl/ParquetBlockInputFormat.h | 3 +- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index a3b34d30ed6..550da8adef0 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -16,6 +16,7 @@ #include "ArrowColumnToCHColumn.h" #include "ArrowFieldIndexUtil.h" #include +#include namespace DB { @@ -44,35 +45,36 @@ Chunk ParquetBlockInputFormat::generate() block_missing_values.clear(); if (!file_reader) + { prepareReader(); + /// It may be necessary to add a parameter + file_reader->set_batch_size(8192); + std::vector row_group_indices; + for (int i = 0; i < file_reader->num_row_groups(); ++i) + { + if (!skip_row_groups.contains(i)) + row_group_indices.emplace_back(i); + } + auto read_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, ¤t_record_batch_reader); + if (!read_status.ok()) + throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); + } if (is_stopped) return {}; - while (row_group_current < row_group_total && skip_row_groups.contains(row_group_current)) - ++row_group_current; - - if (row_group_current >= row_group_total) - return res; - - std::shared_ptr table; - - std::unique_ptr<::arrow::RecordBatchReader> rbr; - std::vector row_group_indices { row_group_current }; - arrow::Status get_batch_reader_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, &rbr); - - if (!get_batch_reader_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", - get_batch_reader_status.ToString()); - - arrow::Status read_status = rbr->ReadAll(&table); - - if (!read_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); - - ++row_group_current; - - arrow_column_to_ch_column->arrowTableToCHChunk(res, table, table->num_rows()); + auto batch = current_record_batch_reader->Next(); + if (*batch) + { + auto tmp_table = arrow::Table::FromRecordBatches({*batch}); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *tmp_table, (*tmp_table)->num_rows()); + } + else + { + current_record_batch_reader.reset(); + file_reader.reset(); + return {}; + } /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 25814090587..37878a94dd9 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -8,7 +8,7 @@ namespace parquet::arrow { class FileReader; } -namespace arrow { class Buffer; } +namespace arrow { class Buffer; class RecordBatchReader;} namespace DB { @@ -46,6 +46,7 @@ private: BlockMissingValues block_missing_values; const FormatSettings format_settings; const std::unordered_set & skip_row_groups; + std::shared_ptr current_record_batch_reader; std::atomic is_stopped{0}; }; From 17fc22a21e3b36ece62f84cb024673184fa26466 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 1 Feb 2023 18:29:20 +0800 Subject: [PATCH 25/78] add parquet max_block_size setting --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 12 ++++++++---- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8ddd42dbecf..db44184a77d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,6 +774,7 @@ class IColumn; M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ + M(UInt64, input_format_parquet_max_block_size, 8192, "Max block size for parquet reader.", 0) \ M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a9045733cac..3fcecd23f5b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -116,6 +116,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; + format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 54de8907169..92e499abb10 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -183,6 +183,7 @@ struct FormatSettings bool case_insensitive_column_matching = false; std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; + UInt64 max_block_size = 8192; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 550da8adef0..6f0b6b62ad8 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -47,23 +47,27 @@ Chunk ParquetBlockInputFormat::generate() if (!file_reader) { prepareReader(); - /// It may be necessary to add a parameter - file_reader->set_batch_size(8192); + file_reader->set_batch_size(format_settings.parquet.max_block_size); std::vector row_group_indices; - for (int i = 0; i < file_reader->num_row_groups(); ++i) + for (int i = 0; i < row_group_total; ++i) { if (!skip_row_groups.contains(i)) row_group_indices.emplace_back(i); } auto read_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, ¤t_record_batch_reader); if (!read_status.ok()) - throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); + throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); } if (is_stopped) return {}; auto batch = current_record_batch_reader->Next(); + if (!batch.ok()) + { + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", + batch.status().ToString()); + } if (*batch) { auto tmp_table = arrow::Table::FromRecordBatches({*batch}); From 2a42c1314b7565d55e4778ba575f2abc475e614f Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 1 Feb 2023 09:45:06 -0500 Subject: [PATCH 26/78] Improved code detals according to suggestions --- .gitmodules | 6 ++--- contrib/idxd-config-cmake/CMakeLists.txt | 23 ++++++++++--------- .../idxd-config-cmake/idxd-header/config.h | 2 +- contrib/qpl-cmake/CMakeLists.txt | 2 +- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/.gitmodules b/.gitmodules index ab169cf9f77..07866e77c58 100644 --- a/.gitmodules +++ b/.gitmodules @@ -257,6 +257,9 @@ [submodule "contrib/qpl"] path = contrib/qpl url = https://github.com/intel/qpl +[submodule "contrib/idxd-config"] + path = contrib/idxd-config + url = https://github.com/intel/idxd-config [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash @@ -330,6 +333,3 @@ [submodule "contrib/crc32-vpmsum"] path = contrib/crc32-vpmsum url = https://github.com/antonblanchard/crc32-vpmsum.git -[submodule "contrib/idxd-config"] - path = contrib/idxd-config - url = https://github.com/intel/idxd-config.git diff --git a/contrib/idxd-config-cmake/CMakeLists.txt b/contrib/idxd-config-cmake/CMakeLists.txt index ee981d0a57b..32c841b3096 100644 --- a/contrib/idxd-config-cmake/CMakeLists.txt +++ b/contrib/idxd-config-cmake/CMakeLists.txt @@ -1,26 +1,27 @@ -## Utility library for controlling and configuring Intel® In-Memory Analytics Accelerator (Intel® IAA) available on Next Generation Intel® Xeon® Scalable processors. +## Utility library for controlling and configuring Intel® In-Memory Analytics Accelerator (Intel® IAA). +## This is to build libaccel-config which is only required by libqpl (./contrib/qpl).Hence, If ENABLE_QPL=False, this module should not be build as well. +if (NOT ENABLE_QPL) + message(STATUS "Not using idxd-config due to ENABLE_QPL=False") + return() +endif() + set(LIBACCEL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config") set(LIBACCEL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/idxd-config") set(UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake") set(LIBACCEL_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake/idxd-header") -#set(SYSROOT_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc/usr/include") set(SRCS "${LIBACCEL_SOURCE_DIR}/accfg/lib/libaccfg.c" "${LIBACCEL_SOURCE_DIR}/util/log.c" "${LIBACCEL_SOURCE_DIR}/util/sysfs.c" ) -#"${LIBACCEL_SOURCE_DIR}/util/log.c" -#"${LIBACCEL_SOURCE_DIR}/util/sysfs.c" -add_library(_LIBACCEL ${SRCS}) -#target_link_libraries(_LIBACCEL PRIVATE ch_contrib::zlib) -target_compile_options(_LIBACCEL PRIVATE "-D_GNU_SOURCE") +add_library(_libaccel ${SRCS}) -target_include_directories(_LIBACCEL BEFORE +target_compile_options(_libaccel PRIVATE "-D_GNU_SOURCE") + +target_include_directories(_libaccel BEFORE PRIVATE ${UUID_DIR} PRIVATE ${LIBACCEL_HEADER_DIR} PRIVATE ${LIBACCEL_SOURCE_DIR}) -#target_include_directories(_LIBACCEL BEFORE PUBLIC ${LIBACCEL_BINARY_DIR}) - -add_library(ch_contrib::LIBACCEL ALIAS _LIBACCEL) +add_library(ch_contrib::libaccel ALIAS _libaccel) diff --git a/contrib/idxd-config-cmake/idxd-header/config.h b/contrib/idxd-config-cmake/idxd-header/config.h index e133ee1a56a..f03b0eac0b0 100644 --- a/contrib/idxd-config-cmake/idxd-header/config.h +++ b/contrib/idxd-config-cmake/idxd-header/config.h @@ -8,7 +8,7 @@ /* #undef ENABLE_DEBUG */ /* Documentation / man pages. */ -#define ENABLE_DOCS 1 +/* #define ENABLE_DOCS */ /* System logging. */ #define ENABLE_LOGGING 1 diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index 8ac2d7bdf9b..aab07842d00 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -316,7 +316,7 @@ target_compile_definitions(_qpl PUBLIC -DENABLE_QPL_COMPRESSION) target_link_libraries(_qpl - PRIVATE ch_contrib::LIBACCEL + PRIVATE ch_contrib::libaccel PRIVATE ${CMAKE_DL_LIBS}) add_library (ch_contrib::qpl ALIAS _qpl) From 3e1b17e4cf525c31a8a6118e8498565f5ab4ce8c Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 1 Feb 2023 17:09:43 +0100 Subject: [PATCH 27/78] fix build --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 48a47ff21c0..a36b3b369a4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1367,7 +1367,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge)) { constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after."; - out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge;); + out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge); return false; } } From bbf060bf89193ad700fc5a780451dbce0c377e27 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 Feb 2023 18:22:05 +0100 Subject: [PATCH 28/78] Fix test --- docs/en/engines/table-engines/integrations/rabbitmq.md | 5 ++++- src/Storages/RabbitMQ/RabbitMQSettings.h | 3 +++ src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 9 ++++----- tests/integration/test_storage_rabbitmq/test.py | 1 + 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 74e9e96c260..eec8691a165 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -77,9 +77,12 @@ Optional parameters: - `rabbitmq_password` - RabbitMQ password. - `rabbitmq_commit_on_select` - Commit messages when select query is made. Default: `false`. - `rabbitmq_max_rows_per_message` — The maximum number of rows written in one RabbitMQ message for row-based formats. Default : `1`. +- `rabbitmq_empty_queue_backoff_start` — A start backoff point to reschedule read if the rabbitmq queue is empty. +- `rabbitmq_empty_queue_backoff_end` — An end backoff point to reschedule read if the rabbitmq queue is empty. -SSL connection: + + * [ ] SSL connection: Use either `rabbitmq_secure = 1` or `amqps` in connection address: `rabbitmq_address = 'amqps://guest:guest@localhost/vhost'`. The default behaviour of the used library is not to check if the created TLS connection is sufficiently secure. Whether the certificate is expired, self-signed, missing or invalid: the connection is simply permitted. More strict checking of certificates can possibly be implemented in the future. diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index f8b126bfe3c..09766b054a1 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -27,6 +27,9 @@ namespace DB M(UInt64, rabbitmq_flush_interval_ms, 0, "Timeout for flushing data from RabbitMQ.", 0) \ M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \ M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \ + M(UInt64, rabbitmq_empty_queue_backoff_start_ms, 10, "A minimum backoff point to reschedule read if the rabbitmq queue is empty", 0) \ + M(UInt64, rabbitmq_empty_queue_backoff_end_ms, 10000, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \ + M(UInt64, rabbitmq_empty_queue_backoff_step_ms, 100, "A maximum backoff point to reschedule read if the rabbitmq queue is empty", 0) \ M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \ M(String, rabbitmq_username, "", "RabbitMQ username", 0) \ M(String, rabbitmq_password, "", "RabbitMQ password", 0) \ diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 80948bdd9c0..f10a60419d1 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -36,7 +36,6 @@ namespace DB static const uint32_t QUEUE_SIZE = 100000; static const auto MAX_FAILED_READ_ATTEMPTS = 10; static const auto RESCHEDULE_MS = 500; -static const auto BACKOFF_TRESHOLD = 32000; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -90,7 +89,7 @@ StorageRabbitMQ::StorageRabbitMQ( , semaphore(0, static_cast(num_consumers)) , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) - , milliseconds_to_wait(RESCHEDULE_MS) + , milliseconds_to_wait(rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms) , is_attach(is_attach_) { const auto & config = getContext()->getConfigRef(); @@ -1021,14 +1020,14 @@ void StorageRabbitMQ::streamingToViewsFunc() if (streamToViews()) { /// Reschedule with backoff. - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; + if (milliseconds_to_wait < rabbitmq_settings->rabbitmq_empty_queue_backoff_end_ms) + milliseconds_to_wait += rabbitmq_settings->rabbitmq_empty_queue_backoff_step_ms; stopLoopIfNoReaders(); break; } else { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = rabbitmq_settings->rabbitmq_empty_queue_backoff_start_ms; } auto end_time = std::chrono::steady_clock::now(); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3ccca75a9cd..d9c37b8cf78 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2735,6 +2735,7 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): result = instance.query("SELECT * FROM test.view ORDER BY key") if rabbitmq_check_result(result): break + time.sleep(1) rabbitmq_check_result(result, True) From 97b6934ed6e1f7c08e4d4a898b6b826759aeed7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 02:42:21 +0300 Subject: [PATCH 29/78] Update ParquetBlockInputFormat.cpp --- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 6f0b6b62ad8..d2ec3c02eed 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -16,7 +16,7 @@ #include "ArrowColumnToCHColumn.h" #include "ArrowFieldIndexUtil.h" #include -#include + namespace DB { From 03109f942d424e8d0d67bde5484c45b2ed569dc8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Feb 2023 01:02:12 +0100 Subject: [PATCH 30/78] better context for stress tests failures --- docker/test/stress/run.sh | 44 +++++++++++++------ .../util/process_functional_tests_result.py | 5 ++- tests/ci/clickhouse_helper.py | 6 ++- tests/ci/report.py | 3 +- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index aa242bfa98d..8347f67fed6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -13,15 +13,28 @@ sysctl kernel.core_pattern='core.%e.%p-%P' OK="\tOK\t\\N\t" FAIL="\tFAIL\t\\N\t" + +FAILURE_CONTEXT_LINES=50 +FAILURE_CONTEXT_MAX_LINE_WIDTH=400 + function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. - clickhouse local -S 's String' --input-format=LineAsString -q "select * from table format CustomSeparated settings format_custom_row_after_delimiter='\\\\\\\\n'" + # Also limit lines width just in case (too long lines are not really useful usually) + clickhouse local -S 's String' --input-format=LineAsString -q "select substr(s, 1, $FAILURE_CONTEXT_MAX_LINE_WIDTH) + from table format CustomSeparated settings format_custom_row_after_delimiter='\\\\\\\\n'" } - function head_escaped() { - head -50 $1 | escaped + head -n $FAILURE_CONTEXT_LINES $1 | escaped +} +function unts() +{ + grep -Po "[0-9][0-9]:[0-9][0-9] \K.*" +} +function trim_server_logs() +{ + head -n $FAILURE_CONTEXT_LINES "/test_output/$1" | grep -Eo " \[ [0-9]+ \] \{.*" | escaped } function install_packages() @@ -167,7 +180,7 @@ function start() then echo "Cannot start clickhouse-server" rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt ||: - echo -e "Cannot start clickhouse-server$FAIL$(head_escaped /test_output/application_errors.txt)" >> /test_output/test_results.tsv + echo -e "Cannot start clickhouse-server$FAIL$(trim_server_logs application_errors.txt)" >> /test_output/test_results.tsv cat /var/log/clickhouse-server/stdout.log tail -n100 /var/log/clickhouse-server/stderr.log tail -n100000 /var/log/clickhouse-server/clickhouse-server.log | rg -F -v -e ' RaftInstance:' -e ' RaftInstance' | tail -n100 @@ -389,7 +402,7 @@ start # NOTE Hung check is implemented in docker/tests/stress/stress rg -Fa "No queries hung" /test_output/test_results.tsv | grep -Fa "OK" \ - || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log)" + || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log | unts)" stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.stress.log @@ -402,7 +415,7 @@ start clickhouse-client --query "SELECT 'Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \ || (rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt \ - && echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(head_escaped /test_output/application_errors.txt)" \ + && echo -e "Server failed to start (see application_errors.txt and clickhouse-server.clean.log)$FAIL$(trim_server_logs application_errors.txt)" \ >> /test_output/test_results.tsv) stop @@ -435,7 +448,7 @@ rg -Fa "Code: 49. DB::Exception: " /var/log/clickhouse-server/clickhouse-server* # No such key errors rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log > /test_output/no_such_key_errors.txt \ - && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(head_escaped /test_output/no_such_key_errors.txt)" >> /test_output/test_results.tsv \ + && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \ || echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv # Remove file no_such_key_errors.txt if it's empty @@ -448,7 +461,7 @@ rg -Fa "########################################" /var/log/clickhouse-server/cli # It also checks for crash without stacktrace (printed by watchdog) rg -Fa " " /var/log/clickhouse-server/clickhouse-server*.log > /test_output/fatal_messages.txt \ - && echo -e "Fatal message in clickhouse-server.log (see fatal_messages.txt)$FAIL$(head_escaped /test_output/fatal_messages.txt)" >> /test_output/test_results.tsv \ + && echo -e "Fatal message in clickhouse-server.log (see fatal_messages.txt)$FAIL$(trim_server_logs fatal_messages.txt)" >> /test_output/test_results.tsv \ || echo -e "No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv # Remove file fatal_messages.txt if it's empty @@ -457,8 +470,13 @@ rg -Fa " " /var/log/clickhouse-server/clickhouse-server*.log > /test_out rg -Fa "########################################" /test_output/* > /dev/null \ && echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv +function get_gdb_log_context() +{ + rg -A50 -Fa " received signal " /test_output/gdb.log | head_escaped +} + rg -Fa " received signal " /test_output/gdb.log > /dev/null \ - && echo -e "Found signal in gdb.log$FAIL$(rg -A50 -Fa " received signal " /test_output/gdb.log | escaped)" >> /test_output/test_results.tsv + && echo -e "Found signal in gdb.log$FAIL$(get_gdb_log_context)" >> /test_output/test_results.tsv if [ "$DISABLE_BC_CHECK" -ne "1" ]; then echo -e "Backward compatibility check\n" @@ -579,7 +597,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then start 500 clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK', NULL, ''" >> /test_output/test_results.tsv \ || (rg --text ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt \ - && echo -e "Backward compatibility check: Server failed to start$FAIL$(head_escaped /test_output/bc_check_application_errors.txt)" >> /test_output/test_results.tsv) + && echo -e "Backward compatibility check: Server failed to start$FAIL$(trim_server_logs bc_check_application_errors.txt)" >> /test_output/test_results.tsv) clickhouse-client --query="SELECT 'Server version: ', version()" @@ -634,7 +652,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then -e "Session expired" \ -e "TOO_MANY_PARTS" \ /var/log/clickhouse-server/clickhouse-server.backward.dirty.log | rg -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(head_escaped /test_output/bc_check_error_messages.txt)" \ + && echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(trim_server_logs bc_check_error_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No Error messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv @@ -657,7 +675,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then # Logical errors echo "Check for Logical errors in server log:" rg -Fa -A20 "Code: 49. DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \ - && echo -e "Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)$FAIL$(head_escaped /test_output/bc_check_logical_errors.txt)" \ + && echo -e "Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)$FAIL$(trim_server_logs bc_check_logical_errors.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No logical errors$OK" >> /test_output/test_results.tsv @@ -672,7 +690,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" rg -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e "Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)$FAIL$(head_escaped /test_output/bc_check_fatal_messages.txt)" \ + && echo -e "Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)$FAIL$(trim_server_logs bc_check_fatal_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "Backward compatibility check: No fatal messages in clickhouse-server.log$OK" >> /test_output/test_results.tsv diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 28f3e211157..257d1587750 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -85,8 +85,11 @@ def process_test_log(log_path): if DATABASE_SIGN in line: test_end = True + # Python does not support TSV, so we have to escape '\t' and '\n' manually + # and hope that complex escape sequences will not break anything test_results = [ - (test[0], test[1], test[2], "".join(test[3])[:4096]) for test in test_results + (test[0], test[1], test[2], + "".join(test[3])[:4096].replace('\t', '\\t').replace('\n', '\\n')) for test in test_results ] return ( diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 654cd6869dc..c873f129468 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -183,7 +183,11 @@ def prepare_tests_results_for_clickhouse( current_row["test_duration_ms"] = int(test_time * 1000) current_row["test_name"] = test_name current_row["test_status"] = test_status - current_row["test_context_raw"] = test_result.raw_logs or "" + if test_result.raw_logs: + # Protect from too big blobs that contain garbage + current_row["test_context_raw"] = test_result.raw_logs[:32 * 1024] + else: + current_row["test_context_raw"] = "" result.append(current_row) return result diff --git a/tests/ci/report.py b/tests/ci/report.py index d33ed4a9d91..0096294f66a 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -224,7 +224,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # The value can be emtpy, but when it's not, # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: - result.set_raw_logs(line[3]) + # Python does not support TSV, so we unescape manually + result.set_raw_logs(line[3]).replace('\\t', '\t').replace('\\n', '\n') else: result.set_log_files(line[3]) From ddbc8ceee8994938c1831a36519db48e8d34c5a5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Feb 2023 00:59:49 +0000 Subject: [PATCH 31/78] Automatic style fix --- docker/test/util/process_functional_tests_result.py | 9 +++++++-- tests/ci/clickhouse_helper.py | 2 +- tests/ci/report.py | 4 +++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 257d1587750..da58db8e45d 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -88,8 +88,13 @@ def process_test_log(log_path): # Python does not support TSV, so we have to escape '\t' and '\n' manually # and hope that complex escape sequences will not break anything test_results = [ - (test[0], test[1], test[2], - "".join(test[3])[:4096].replace('\t', '\\t').replace('\n', '\\n')) for test in test_results + ( + test[0], + test[1], + test[2], + "".join(test[3])[:4096].replace("\t", "\\t").replace("\n", "\\n"), + ) + for test in test_results ] return ( diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index c873f129468..d60a9e6afd1 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -185,7 +185,7 @@ def prepare_tests_results_for_clickhouse( current_row["test_status"] = test_status if test_result.raw_logs: # Protect from too big blobs that contain garbage - current_row["test_context_raw"] = test_result.raw_logs[:32 * 1024] + current_row["test_context_raw"] = test_result.raw_logs[: 32 * 1024] else: current_row["test_context_raw"] = "" result.append(current_row) diff --git a/tests/ci/report.py b/tests/ci/report.py index 0096294f66a..e1601cab805 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,7 +225,9 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3]).replace('\\t', '\t').replace('\\n', '\n') + result.set_raw_logs(line[3]).replace("\\t", "\t").replace( + "\\n", "\n" + ) else: result.set_log_files(line[3]) From 8e684170fe6bf86f1d5c6421319a6f3b842958d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 02:11:16 +0100 Subject: [PATCH 32/78] Fix even more trash --- src/Parsers/ASTAlterNamedCollectionQuery.h | 2 ++ src/Parsers/ASTBackupQuery.cpp | 5 ++++ src/Parsers/ASTBackupQuery.h | 1 + src/Parsers/ASTCheckQuery.h | 2 ++ src/Parsers/ASTCreateFunctionQuery.h | 2 ++ src/Parsers/ASTCreateNamedCollectionQuery.h | 2 ++ src/Parsers/ASTDeleteQuery.h | 1 + src/Parsers/ASTDropFunctionQuery.h | 2 ++ src/Parsers/ASTDropIndexQuery.cpp | 2 +- src/Parsers/ASTDropNamedCollectionQuery.h | 2 ++ src/Parsers/ASTExplainQuery.h | 2 ++ src/Parsers/ASTExternalDDLQuery.h | 3 +++ src/Parsers/ASTKillQueryQuery.h | 2 ++ src/Parsers/ASTOptimizeQuery.h | 2 ++ src/Parsers/ASTQueryWithTableAndOutput.h | 2 ++ src/Parsers/ASTSetQuery.h | 2 ++ src/Parsers/ASTShowTablesQuery.h | 2 ++ src/Parsers/ASTTransactionControl.cpp | 15 +++++++++++ src/Parsers/ASTTransactionControl.h | 2 ++ src/Parsers/ASTUseQuery.h | 2 ++ src/Parsers/ASTWatchQuery.h | 2 ++ src/Parsers/Access/ASTCreateQuotaQuery.h | 2 ++ src/Parsers/Access/ASTCreateRoleQuery.h | 2 ++ src/Parsers/Access/ASTCreateRowPolicyQuery.h | 2 ++ .../Access/ASTCreateSettingsProfileQuery.h | 1 + src/Parsers/Access/ASTCreateUserQuery.h | 2 ++ src/Parsers/Access/ASTDropAccessEntityQuery.h | 2 ++ src/Parsers/Access/ASTSetRoleQuery.h | 2 ++ .../Access/ASTShowAccessEntitiesQuery.h | 2 ++ .../Access/ASTShowCreateAccessEntityQuery.h | 2 ++ src/Parsers/Access/ASTShowGrantsQuery.h | 2 ++ src/Parsers/IAST.h | 25 ++++++++++++++++--- src/Parsers/TablePropertiesQueriesASTs.h | 4 +++ 33 files changed, 100 insertions(+), 5 deletions(-) diff --git a/src/Parsers/ASTAlterNamedCollectionQuery.h b/src/Parsers/ASTAlterNamedCollectionQuery.h index a8aa06200fd..36cf9c65b81 100644 --- a/src/Parsers/ASTAlterNamedCollectionQuery.h +++ b/src/Parsers/ASTAlterNamedCollectionQuery.h @@ -23,6 +23,8 @@ public: void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Alter; } }; } diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 85a0a0c7799..2c26e723687 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -289,4 +289,9 @@ ASTPtr ASTBackupQuery::getRewrittenASTWithoutOnCluster(const WithoutOnClusterAST return new_query; } +IAST::QueryKind ASTBackupQuery::getQueryKind() const +{ + return kind == Kind::BACKUP ? QueryKind::Backup : QueryKind::Restore; +} + } diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index 708f5bf468f..a3e3a144c72 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -93,5 +93,6 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override; + QueryKind getQueryKind() const override; }; } diff --git a/src/Parsers/ASTCheckQuery.h b/src/Parsers/ASTCheckQuery.h index 8b376ef8d60..f29a0bd5406 100644 --- a/src/Parsers/ASTCheckQuery.h +++ b/src/Parsers/ASTCheckQuery.h @@ -23,6 +23,8 @@ struct ASTCheckQuery : public ASTQueryWithTableAndOutput return res; } + QueryKind getQueryKind() const override { return QueryKind::Check; } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { diff --git a/src/Parsers/ASTCreateFunctionQuery.h b/src/Parsers/ASTCreateFunctionQuery.h index ce0d874b15d..8ce167ba7db 100644 --- a/src/Parsers/ASTCreateFunctionQuery.h +++ b/src/Parsers/ASTCreateFunctionQuery.h @@ -25,6 +25,8 @@ public: ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } String getFunctionName() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/ASTCreateNamedCollectionQuery.h b/src/Parsers/ASTCreateNamedCollectionQuery.h index 901e6b50a4c..303d1901931 100644 --- a/src/Parsers/ASTCreateNamedCollectionQuery.h +++ b/src/Parsers/ASTCreateNamedCollectionQuery.h @@ -22,6 +22,8 @@ public: ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + QueryKind getQueryKind() const override { return QueryKind::Create; } + std::string getCollectionName() const; }; diff --git a/src/Parsers/ASTDeleteQuery.h b/src/Parsers/ASTDeleteQuery.h index bcb97639b64..3033a89db3c 100644 --- a/src/Parsers/ASTDeleteQuery.h +++ b/src/Parsers/ASTDeleteQuery.h @@ -11,6 +11,7 @@ class ASTDeleteQuery : public ASTQueryWithTableAndOutput public: String getID(char delim) const final; ASTPtr clone() const final; + QueryKind getQueryKind() const override { return QueryKind::Delete; } ASTPtr predicate; diff --git a/src/Parsers/ASTDropFunctionQuery.h b/src/Parsers/ASTDropFunctionQuery.h index edfa6a23994..da8fb1ba1c1 100644 --- a/src/Parsers/ASTDropFunctionQuery.h +++ b/src/Parsers/ASTDropFunctionQuery.h @@ -21,6 +21,8 @@ public: void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } }; } diff --git a/src/Parsers/ASTDropIndexQuery.cpp b/src/Parsers/ASTDropIndexQuery.cpp index a07336a2d26..43de582ba8a 100644 --- a/src/Parsers/ASTDropIndexQuery.cpp +++ b/src/Parsers/ASTDropIndexQuery.cpp @@ -9,7 +9,7 @@ namespace DB /** Get the text that identifies this element. */ String ASTDropIndexQuery::getID(char delim) const { - return "CreateIndexQuery" + (delim + getDatabase()) + delim + getTable(); + return "DropIndexQuery" + (delim + getDatabase()) + delim + getTable(); } ASTPtr ASTDropIndexQuery::clone() const diff --git a/src/Parsers/ASTDropNamedCollectionQuery.h b/src/Parsers/ASTDropNamedCollectionQuery.h index 0b71bdaf213..2ead6c72532 100644 --- a/src/Parsers/ASTDropNamedCollectionQuery.h +++ b/src/Parsers/ASTDropNamedCollectionQuery.h @@ -20,6 +20,8 @@ public: void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } }; } diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index cb8b3199c81..3903cf42269 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -109,6 +109,8 @@ public: const ASTPtr & getTableFunction() const { return table_function; } const ASTPtr & getTableOverride() const { return table_override; } + QueryKind getQueryKind() const override { return QueryKind::Explain; } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { diff --git a/src/Parsers/ASTExternalDDLQuery.h b/src/Parsers/ASTExternalDDLQuery.h index 4ec68aa8b9f..7913d44b970 100644 --- a/src/Parsers/ASTExternalDDLQuery.h +++ b/src/Parsers/ASTExternalDDLQuery.h @@ -4,6 +4,7 @@ #include #include + namespace DB { @@ -38,6 +39,8 @@ public: from->formatImpl(settings, state, stacked); external_ddl->formatImpl(settings, state, stacked); } + + QueryKind getQueryKind() const override { return QueryKind::ExternalDDL; } }; } diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index 88a1be38766..20db5576fa4 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -42,6 +42,8 @@ public: { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::KillQuery; } }; } diff --git a/src/Parsers/ASTOptimizeQuery.h b/src/Parsers/ASTOptimizeQuery.h index c53a61a66dc..67721da4176 100644 --- a/src/Parsers/ASTOptimizeQuery.h +++ b/src/Parsers/ASTOptimizeQuery.h @@ -54,6 +54,8 @@ public: { return removeOnCluster(clone(), params.default_database); } + + QueryKind getQueryKind() const override { return QueryKind::Optimize; } }; } diff --git a/src/Parsers/ASTQueryWithTableAndOutput.h b/src/Parsers/ASTQueryWithTableAndOutput.h index 233028c5023..10f0e76f29c 100644 --- a/src/Parsers/ASTQueryWithTableAndOutput.h +++ b/src/Parsers/ASTQueryWithTableAndOutput.h @@ -49,6 +49,8 @@ public: return res; } + QueryKind getQueryKind() const override { return QueryKind::Show; } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { diff --git a/src/Parsers/ASTSetQuery.h b/src/Parsers/ASTSetQuery.h index 0e1933cbf1a..40abe2de31d 100644 --- a/src/Parsers/ASTSetQuery.h +++ b/src/Parsers/ASTSetQuery.h @@ -35,6 +35,8 @@ public: void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; void updateTreeHashImpl(SipHash & hash_state) const override; + + QueryKind getQueryKind() const override { return QueryKind::Set; } }; } diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index c3e7f0799d4..b58d65e37ab 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -39,6 +39,8 @@ public: ASTPtr clone() const override; + QueryKind getQueryKind() const override { return QueryKind::Show; } + protected: void formatLike(const FormatSettings & settings) const; void formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; diff --git a/src/Parsers/ASTTransactionControl.cpp b/src/Parsers/ASTTransactionControl.cpp index 3ff29d9e43e..3106d432c90 100644 --- a/src/Parsers/ASTTransactionControl.cpp +++ b/src/Parsers/ASTTransactionControl.cpp @@ -24,6 +24,21 @@ void ASTTransactionControl::formatImpl(const FormatSettings & format /*state*/, } } +IAST::QueryKind ASTTransactionControl::getQueryKind() const +{ + switch (action) + { + case BEGIN: + return QueryKind::Begin; + case COMMIT: + return QueryKind::Commit; + case ROLLBACK: + return QueryKind::Rollback; + case SET_SNAPSHOT: + return QueryKind::SetTransactionSnapshot; + } +} + void ASTTransactionControl::updateTreeHashImpl(SipHash & hash_state) const { hash_state.update(action); diff --git a/src/Parsers/ASTTransactionControl.h b/src/Parsers/ASTTransactionControl.h index 06f578ff138..fb0058144dd 100644 --- a/src/Parsers/ASTTransactionControl.h +++ b/src/Parsers/ASTTransactionControl.h @@ -27,6 +27,8 @@ public: void formatImpl(const FormatSettings & format, FormatState & /*state*/, FormatStateStacked /*frame*/) const override; void updateTreeHashImpl(SipHash & hash_state) const override; + + QueryKind getQueryKind() const override; }; } diff --git a/src/Parsers/ASTUseQuery.h b/src/Parsers/ASTUseQuery.h index 16d449f905f..f767a6bbdb7 100644 --- a/src/Parsers/ASTUseQuery.h +++ b/src/Parsers/ASTUseQuery.h @@ -21,6 +21,8 @@ public: ASTPtr clone() const override { return std::make_shared(*this); } + QueryKind getQueryKind() const override { return QueryKind::Use; } + protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { diff --git a/src/Parsers/ASTWatchQuery.h b/src/Parsers/ASTWatchQuery.h index ea02c18a51d..29dadd71675 100644 --- a/src/Parsers/ASTWatchQuery.h +++ b/src/Parsers/ASTWatchQuery.h @@ -37,6 +37,8 @@ public: return res; } + QueryKind getQueryKind() const override { return QueryKind::Create; } + protected: void formatQueryImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override { diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index b6add6d8321..f5eb59800ec 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -55,5 +55,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void replaceCurrentUserTag(const String & current_user_name) const; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/Access/ASTCreateRoleQuery.h b/src/Parsers/Access/ASTCreateRoleQuery.h index 906ea683e1a..42d1a4031b6 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.h +++ b/src/Parsers/Access/ASTCreateRoleQuery.h @@ -35,5 +35,7 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index b96cc325524..32d98fab822 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -51,5 +51,7 @@ public: void replaceCurrentUserTag(const String & current_user_name) const; void replaceEmptyDatabase(const String & current_database) const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h index 441ec0f5233..c1a64998f29 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h @@ -41,5 +41,6 @@ public: void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; void replaceCurrentUserTag(const String & current_user_name) const; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 412775d3bf3..a3571dd6c61 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -63,5 +63,7 @@ public: void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; bool hasSecretParts() const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Create; } }; } diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index 7ca672ad989..f3a065c50df 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -29,5 +29,7 @@ public: ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } void replaceEmptyDatabase(const String & current_database) const; + + QueryKind getQueryKind() const override { return QueryKind::Drop; } }; } diff --git a/src/Parsers/Access/ASTSetRoleQuery.h b/src/Parsers/Access/ASTSetRoleQuery.h index f0170ae6af2..1146205af2b 100644 --- a/src/Parsers/Access/ASTSetRoleQuery.h +++ b/src/Parsers/Access/ASTSetRoleQuery.h @@ -27,5 +27,7 @@ public: String getID(char) const override; ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + + QueryKind getQueryKind() const override { return QueryKind::Set; } }; } diff --git a/src/Parsers/Access/ASTShowAccessEntitiesQuery.h b/src/Parsers/Access/ASTShowAccessEntitiesQuery.h index e633a4b506a..9957f8d5705 100644 --- a/src/Parsers/Access/ASTShowAccessEntitiesQuery.h +++ b/src/Parsers/Access/ASTShowAccessEntitiesQuery.h @@ -31,6 +31,8 @@ public: void replaceEmptyDatabase(const String & current_database); + QueryKind getQueryKind() const override { return QueryKind::Show; } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h index 27f13587033..657160a96dd 100644 --- a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h +++ b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h @@ -40,6 +40,8 @@ public: void replaceEmptyDatabase(const String & current_database); + QueryKind getQueryKind() const override { return QueryKind::Show; } + protected: String getKeyword() const; void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/Access/ASTShowGrantsQuery.h b/src/Parsers/Access/ASTShowGrantsQuery.h index 04764fe3502..ceaf2f586cb 100644 --- a/src/Parsers/Access/ASTShowGrantsQuery.h +++ b/src/Parsers/Access/ASTShowGrantsQuery.h @@ -17,5 +17,7 @@ public: String getID(char) const override; ASTPtr clone() const override; void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + + QueryKind getQueryKind() const override { return QueryKind::Show; } }; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 460ecca7ea0..627b1174b33 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -253,15 +253,32 @@ public: enum class QueryKind : uint8_t { None = 0, - Alter, + Select, + Insert, + Delete, Create, Drop, - Grant, - Insert, Rename, + Optimize, + Check, + Alter, + Grant, Revoke, - Select, System, + Set, + Use, + Show, + Exists, + Describe, + Explain, + Backup, + Restore, + KillQuery, + ExternalDDL, + Begin, + Commit, + Rollback, + SetTransactionSnapshot, }; /// Return QueryKind of this AST query. virtual QueryKind getQueryKind() const { return QueryKind::None; } diff --git a/src/Parsers/TablePropertiesQueriesASTs.h b/src/Parsers/TablePropertiesQueriesASTs.h index 3de6feb2409..038936a0297 100644 --- a/src/Parsers/TablePropertiesQueriesASTs.h +++ b/src/Parsers/TablePropertiesQueriesASTs.h @@ -91,6 +91,8 @@ protected: settings.ostr << (settings.hilite ? hilite_keyword : "") << ASTExistsDatabaseQueryIDAndQueryNames::Query << " " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(getDatabase()); } + + QueryKind getQueryKind() const override { return QueryKind::Exists; } }; class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl @@ -123,6 +125,8 @@ public: return res; } + QueryKind getQueryKind() const override { return QueryKind::Describe; } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { From 4089c343373ef18e2847c0af1777142a2e72101c Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Thu, 2 Feb 2023 09:26:27 -0500 Subject: [PATCH 33/78] move the content of idxd-config-cmake into QPL-cmake --- contrib/CMakeLists.txt | 1 - contrib/idxd-config-cmake/CMakeLists.txt | 27 ------------------- contrib/idxd-config-cmake/uuid/uuid.h | 4 --- contrib/qpl-cmake/CMakeLists.txt | 25 ++++++++++++++--- .../idxd-header/config.h | 0 5 files changed, 22 insertions(+), 35 deletions(-) delete mode 100644 contrib/idxd-config-cmake/CMakeLists.txt delete mode 100644 contrib/idxd-config-cmake/uuid/uuid.h rename contrib/{idxd-config-cmake => qpl-cmake}/idxd-header/config.h (100%) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 180e81b7641..f5d1315cc02 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -178,7 +178,6 @@ add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (c-ares-cmake c-ares) add_contrib (qpl-cmake qpl) -add_contrib (idxd-config-cmake idxd-config) add_contrib (morton-nd-cmake morton-nd) if (ARCH_S390X) diff --git a/contrib/idxd-config-cmake/CMakeLists.txt b/contrib/idxd-config-cmake/CMakeLists.txt deleted file mode 100644 index 32c841b3096..00000000000 --- a/contrib/idxd-config-cmake/CMakeLists.txt +++ /dev/null @@ -1,27 +0,0 @@ -## Utility library for controlling and configuring Intel® In-Memory Analytics Accelerator (Intel® IAA). -## This is to build libaccel-config which is only required by libqpl (./contrib/qpl).Hence, If ENABLE_QPL=False, this module should not be build as well. -if (NOT ENABLE_QPL) - message(STATUS "Not using idxd-config due to ENABLE_QPL=False") - return() -endif() - -set(LIBACCEL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config") -set(LIBACCEL_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/idxd-config") -set(UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake") -set(LIBACCEL_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config-cmake/idxd-header") -set(SRCS - "${LIBACCEL_SOURCE_DIR}/accfg/lib/libaccfg.c" - "${LIBACCEL_SOURCE_DIR}/util/log.c" - "${LIBACCEL_SOURCE_DIR}/util/sysfs.c" -) - -add_library(_libaccel ${SRCS}) - -target_compile_options(_libaccel PRIVATE "-D_GNU_SOURCE") - -target_include_directories(_libaccel BEFORE - PRIVATE ${UUID_DIR} - PRIVATE ${LIBACCEL_HEADER_DIR} - PRIVATE ${LIBACCEL_SOURCE_DIR}) - -add_library(ch_contrib::libaccel ALIAS _libaccel) diff --git a/contrib/idxd-config-cmake/uuid/uuid.h b/contrib/idxd-config-cmake/uuid/uuid.h deleted file mode 100644 index bf108ba0d29..00000000000 --- a/contrib/idxd-config-cmake/uuid/uuid.h +++ /dev/null @@ -1,4 +0,0 @@ -#ifndef _QPL_UUID_UUID_H -#define _QPL_UUID_UUID_H -typedef unsigned char uuid_t[16]; -#endif /* _QPL_UUID_UUID_H */ diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index aab07842d00..fc5548b0652 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -10,11 +10,30 @@ if (NOT ENABLE_QPL) return() endif() +## QPL has build dependency on libaccel-config. Here is to build libaccel-config which is required by QPL. +## libaccel-config is the utility library for controlling and configuring Intel® In-Memory Analytics Accelerator (Intel® IAA). +set (LIBACCEL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/idxd-config") +set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") +set (LIBACCEL_HEADER_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake/idxd-header") +set (SRCS + "${LIBACCEL_SOURCE_DIR}/accfg/lib/libaccfg.c" + "${LIBACCEL_SOURCE_DIR}/util/log.c" + "${LIBACCEL_SOURCE_DIR}/util/sysfs.c" +) + +add_library(accel-config ${SRCS}) + +target_compile_options(accel-config PRIVATE "-D_GNU_SOURCE") + +target_include_directories(accel-config BEFORE + PRIVATE ${UUID_DIR} + PRIVATE ${LIBACCEL_HEADER_DIR} + PRIVATE ${LIBACCEL_SOURCE_DIR}) + +## QPL build start here. set (QPL_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl") set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/build/contrib/qpl") -set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") - set (EFFICIENT_WAIT OFF) set (BLOCK_ON_FAULT ON) set (LOG_HW_INIT OFF) @@ -316,7 +335,7 @@ target_compile_definitions(_qpl PUBLIC -DENABLE_QPL_COMPRESSION) target_link_libraries(_qpl - PRIVATE ch_contrib::libaccel + PRIVATE accel-config PRIVATE ${CMAKE_DL_LIBS}) add_library (ch_contrib::qpl ALIAS _qpl) diff --git a/contrib/idxd-config-cmake/idxd-header/config.h b/contrib/qpl-cmake/idxd-header/config.h similarity index 100% rename from contrib/idxd-config-cmake/idxd-header/config.h rename to contrib/qpl-cmake/idxd-header/config.h From 0cbd7497b1fd7cea7bfb1bc17a71fb213af0cf86 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 2 Feb 2023 11:24:34 +0800 Subject: [PATCH 34/78] Under multi-threading, different order result sets may be generated --- .../0_stateless/02481_parquet_int_list_multiple_chunks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02481_parquet_int_list_multiple_chunks.sh b/tests/queries/0_stateless/02481_parquet_int_list_multiple_chunks.sh index c2c6f689851..5c7c9701a67 100755 --- a/tests/queries/0_stateless/02481_parquet_int_list_multiple_chunks.sh +++ b/tests/queries/0_stateless/02481_parquet_int_list_multiple_chunks.sh @@ -37,6 +37,6 @@ DATA_FILE=$CUR_DIR/data_parquet/int-list-zero-based-chunked-array.parquet ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (arr Array(Int64)) ENGINE = Memory" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load SETTINGS max_threads=1" | md5sum ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" ${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file From 11e89d9aa0834451dd16824749b1bd98de4e2a39 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 04:40:45 +0100 Subject: [PATCH 35/78] Add a test --- .../test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py index 97f39c60004..777c35f8b50 100644 --- a/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py +++ b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py @@ -96,6 +96,16 @@ def test_select(started_cluster): 10, ) + # intersect and except are counted + common_pattern( + node_select, + "select", + "select sleep(1) INTERSECT select sleep(1) EXCEPT select sleep(1)", + "insert into test_concurrent_insert values (0)", + 2, + 10, + ) + def test_insert(started_cluster): common_pattern( From 8513e5a721ad0178a0a798ce516fbe062ad6287d Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 08:48:13 +0000 Subject: [PATCH 36/78] Fix tupleElement with Null arguments --- src/Functions/tupleElement.cpp | 7 +++---- .../02541_tuple_element_with_null.reference | 2 ++ .../02541_tuple_element_with_null.sql | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02541_tuple_element_with_null.reference create mode 100644 tests/queries/0_stateless/02541_tuple_element_with_null.sql diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 829262de30a..879b6feed40 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -56,10 +56,9 @@ public: return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override - { - return {1}; - } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + bool useDefaultImplementationForNulls() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/tests/queries/0_stateless/02541_tuple_element_with_null.reference b/tests/queries/0_stateless/02541_tuple_element_with_null.reference new file mode 100644 index 00000000000..7afa6248576 --- /dev/null +++ b/tests/queries/0_stateless/02541_tuple_element_with_null.reference @@ -0,0 +1,2 @@ +1 1 2 2 +\N \N 3 3 diff --git a/tests/queries/0_stateless/02541_tuple_element_with_null.sql b/tests/queries/0_stateless/02541_tuple_element_with_null.sql new file mode 100644 index 00000000000..d2062b60d49 --- /dev/null +++ b/tests/queries/0_stateless/02541_tuple_element_with_null.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test_tuple_element; +CREATE TABLE test_tuple_element +( + tuple Tuple(k1 Nullable(UInt64), k2 UInt64) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS index_granularity = 8192; + +INSERT INTO test_tuple_element VALUES (tuple(1,2)), (tuple(NULL, 3)); + +SELECT + tupleElement(tuple, 'k1', 0) fine_k1_with_0, + tupleElement(tuple, 'k1', NULL) k1_with_null, + tupleElement(tuple, 'k2', 0) k2_with_0, + tupleElement(tuple, 'k2', NULL) k2_with_null +FROM test_tuple_element; + +DROP TABLE test_tuple_element; From f7665e136aa19ed3f7fc65d140d38c48754d8afc Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 09:34:42 +0000 Subject: [PATCH 37/78] update test update test --- .../queries/0_stateless/01710_minmax_count_projection.reference | 1 - tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index bbcec98fb74..4dd46ccd22c 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -16,7 +16,6 @@ 1 1 1 -2021-10-25 10:00:00 3 2021-10-27 10:00:00 3 \N 2021-10-27 10:00:00 3 0 2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index f01e5915393..1967d9fe53c 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -58,7 +58,7 @@ select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(d select count() from d group by toDate(dt); -- fuzz crash -SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; +SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- { serverError 43 } -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; From f1b4ee059c2dc46ceb8508d2a2a3b8840aa79fd7 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 2 Feb 2023 10:34:47 +0000 Subject: [PATCH 38/78] update test --- .../queries/0_stateless/01710_minmax_count_projection.reference | 1 + tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 4dd46ccd22c..bbcec98fb74 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -16,6 +16,7 @@ 1 1 1 +2021-10-25 10:00:00 3 2021-10-27 10:00:00 3 \N 2021-10-27 10:00:00 3 0 2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 1967d9fe53c..f7645414187 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -58,7 +58,7 @@ select min(dt), max(dt), count(toDate(dt) >= '2021-10-25') from d where toDate(d select count() from d group by toDate(dt); -- fuzz crash -SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- { serverError 43 } +SELECT min(dt), count(ignore(ignore(ignore(tupleElement(_partition_value, 'xxxx', NULL) = NULL), NULL, NULL, NULL), 0, '10485.76', NULL)), max(dt), count(toDate(dt) >= '2021-10-25') FROM d WHERE toDate(dt) >= '2021-10-25'; -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; From d85ad3c2191d3448a499cad8716e1d66ca76979d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 Feb 2023 12:15:29 +0100 Subject: [PATCH 39/78] Fix tests --- tests/integration/test_storage_rabbitmq/test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index d9c37b8cf78..31799e3a7aa 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -377,10 +377,11 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): connection = pika.BlockingConnection(parameters) channel = connection.channel() + instance.wait_for_log_line("Started streaming to 2 attached views") + messages = [] for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - for message in messages: + message = json.dumps({"key": i, "value": i}) channel.basic_publish(exchange="mv", routing_key="", body=message) time_limit_sec = 60 @@ -397,8 +398,10 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): while time.monotonic() < deadline: result = instance.query("SELECT * FROM test.view2 ORDER BY key") + print(f"Result: {result}") if rabbitmq_check_result(result): break + time.sleep(1) rabbitmq_check_result(result, True) connection.close() @@ -480,6 +483,8 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): connection = pika.BlockingConnection(parameters) channel = connection.channel() + instance.wait_for_log_line("Started streaming to 2 attached views") + messages = [] for i in range(50): messages.append(json.dumps({"key": i, "value": i})) From 5e64cda339bfaf650d353af688ff0a91f1192799 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Feb 2023 15:28:01 +0300 Subject: [PATCH 40/78] Update tests/ci/report.py Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com> --- tests/ci/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index e1601cab805..5422c463fb8 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,8 +225,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3]).replace("\\t", "\t").replace( - "\\n", "\n" + result.set_raw_logs(line[3].replace("\\t", "\t").replace( + "\\n", "\n") ) else: result.set_log_files(line[3]) From 892a3ff316e44061e2567998d1cebb3ba14f4522 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Feb 2023 12:40:50 +0000 Subject: [PATCH 41/78] Automatic style fix --- tests/ci/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 5422c463fb8..947fb33d905 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -225,8 +225,8 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes # the 4th value is a pythonic list, e.g. ['file1', 'file2'] if with_raw_logs: # Python does not support TSV, so we unescape manually - result.set_raw_logs(line[3].replace("\\t", "\t").replace( - "\\n", "\n") + result.set_raw_logs( + line[3].replace("\\t", "\t").replace("\\n", "\n") ) else: result.set_log_files(line[3]) From 3472ed4f0d9a8396a5d4fb21d8dcb026341c1a79 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Feb 2023 15:47:00 +0000 Subject: [PATCH 42/78] Make some classes 'final' --- src/BridgeHelper/CatBoostLibraryBridgeHelper.h | 2 +- src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.h | 2 +- src/Compression/CachedCompressedReadBuffer.h | 2 +- src/Compression/CheckingCompressedReadBuffer.h | 2 +- src/Compression/CompressedReadBuffer.h | 2 +- src/Compression/CompressedReadBufferFromFile.h | 2 +- src/Compression/CompressedWriteBuffer.h | 2 +- src/Compression/CompressionCodecDeflateQpl.h | 2 +- src/Compression/CompressionCodecEncrypted.h | 2 +- src/Compression/CompressionCodecNone.h | 2 +- src/Processors/ConcatProcessor.h | 2 +- src/Processors/DelayedPortsProcessor.h | 2 +- src/Processors/ForkProcessor.h | 2 +- src/Processors/LimitTransform.h | 2 +- src/Processors/OffsetTransform.h | 2 +- src/Processors/QueueBuffer.h | 2 +- src/Processors/ResizeProcessor.h | 2 +- 17 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/BridgeHelper/CatBoostLibraryBridgeHelper.h b/src/BridgeHelper/CatBoostLibraryBridgeHelper.h index 91c94143147..55dfd715f00 100644 --- a/src/BridgeHelper/CatBoostLibraryBridgeHelper.h +++ b/src/BridgeHelper/CatBoostLibraryBridgeHelper.h @@ -11,7 +11,7 @@ namespace DB { -class CatBoostLibraryBridgeHelper : public LibraryBridgeHelper +class CatBoostLibraryBridgeHelper final : public LibraryBridgeHelper { public: static constexpr inline auto PING_HANDLER = "/catboost_ping"; diff --git a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.h b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.h index feebf997387..5632fd2a28e 100644 --- a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.h +++ b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.h @@ -14,7 +14,7 @@ namespace DB class Pipe; // Class to access the external dictionary part of the clickhouse-library-bridge. -class ExternalDictionaryLibraryBridgeHelper : public LibraryBridgeHelper +class ExternalDictionaryLibraryBridgeHelper final : public LibraryBridgeHelper { public: diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 4493a40156c..ea8d5dbfd99 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -17,7 +17,7 @@ namespace DB * Disadvantages: * - in case you need to read a lot of data in a row, but some of them only a part is cached, you have to do seek-and. */ -class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +class CachedCompressedReadBuffer final : public CompressedReadBufferBase, public ReadBuffer { private: std::function()> file_in_creator; diff --git a/src/Compression/CheckingCompressedReadBuffer.h b/src/Compression/CheckingCompressedReadBuffer.h index fcb222eddc3..5bbf7789ec9 100644 --- a/src/Compression/CheckingCompressedReadBuffer.h +++ b/src/Compression/CheckingCompressedReadBuffer.h @@ -11,7 +11,7 @@ namespace DB /** A buffer for reading from a compressed file with just checking checksums of * the compressed blocks, without any decompression. */ -class CheckingCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +class CheckingCompressedReadBuffer final : public CompressedReadBufferBase, public ReadBuffer { protected: bool nextImpl() override; diff --git a/src/Compression/CompressedReadBuffer.h b/src/Compression/CompressedReadBuffer.h index 1d338303c84..0c537d171c4 100644 --- a/src/Compression/CompressedReadBuffer.h +++ b/src/Compression/CompressedReadBuffer.h @@ -8,7 +8,7 @@ namespace DB { -class CompressedReadBuffer : public CompressedReadBufferBase, public BufferWithOwnMemory +class CompressedReadBuffer final : public CompressedReadBufferBase, public BufferWithOwnMemory { private: size_t size_compressed = 0; diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index d307503fb99..e52b7378a1b 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -14,7 +14,7 @@ class MMappedFileCache; /// Unlike CompressedReadBuffer, it can do seek. -class CompressedReadBufferFromFile : public CompressedReadBufferBase, public BufferWithOwnMemory +class CompressedReadBufferFromFile final : public CompressedReadBufferBase, public BufferWithOwnMemory { private: /** At any time, one of two things is true: diff --git a/src/Compression/CompressedWriteBuffer.h b/src/Compression/CompressedWriteBuffer.h index 86c6762764b..6ae1fbee9cc 100644 --- a/src/Compression/CompressedWriteBuffer.h +++ b/src/Compression/CompressedWriteBuffer.h @@ -13,7 +13,7 @@ namespace DB { -class CompressedWriteBuffer : public BufferWithOwnMemory +class CompressedWriteBuffer final : public BufferWithOwnMemory { public: explicit CompressedWriteBuffer( diff --git a/src/Compression/CompressionCodecDeflateQpl.h b/src/Compression/CompressionCodecDeflateQpl.h index 1a13a1ca42c..3171a898311 100644 --- a/src/Compression/CompressionCodecDeflateQpl.h +++ b/src/Compression/CompressionCodecDeflateQpl.h @@ -87,7 +87,7 @@ private: Poco::Logger * log; }; -class CompressionCodecDeflateQpl : public ICompressionCodec +class CompressionCodecDeflateQpl final : public ICompressionCodec { public: CompressionCodecDeflateQpl(); diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index 763e4da5d56..0f680dbcb09 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -44,7 +44,7 @@ enum EncryptionMethod * as otherwise our engines like ReplicatedMergeTree cannot * deduplicate data blocks. */ -class CompressionCodecEncrypted : public ICompressionCodec +class CompressionCodecEncrypted final : public ICompressionCodec { public: /** If a key is available, the server is supposed to diff --git a/src/Compression/CompressionCodecNone.h b/src/Compression/CompressionCodecNone.h index bf6bb6de4e2..1565720947d 100644 --- a/src/Compression/CompressionCodecNone.h +++ b/src/Compression/CompressionCodecNone.h @@ -8,7 +8,7 @@ namespace DB { -class CompressionCodecNone : public ICompressionCodec +class CompressionCodecNone final : public ICompressionCodec { public: CompressionCodecNone(); diff --git a/src/Processors/ConcatProcessor.h b/src/Processors/ConcatProcessor.h index 4a1fc580411..506317eba5d 100644 --- a/src/Processors/ConcatProcessor.h +++ b/src/Processors/ConcatProcessor.h @@ -13,7 +13,7 @@ namespace DB * Doesn't do any heavy calculations. * Preserves an order of data. */ -class ConcatProcessor : public IProcessor +class ConcatProcessor final : public IProcessor { public: ConcatProcessor(const Block & header, size_t num_inputs); diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index a6a9590e0c8..3909d533914 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -8,7 +8,7 @@ namespace DB /// Some ports are delayed. Delayed ports are processed after other outputs are all finished. /// Data between ports is not mixed. It is important because this processor can be used before MergingSortedTransform. /// Delayed ports are appeared after joins, when some non-matched data need to be processed at the end. -class DelayedPortsProcessor : public IProcessor +class DelayedPortsProcessor final : public IProcessor { public: DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports, bool assert_main_ports_empty = false); diff --git a/src/Processors/ForkProcessor.h b/src/Processors/ForkProcessor.h index 8839f73584f..c6a2a57c657 100644 --- a/src/Processors/ForkProcessor.h +++ b/src/Processors/ForkProcessor.h @@ -15,7 +15,7 @@ namespace DB * Doesn't do any heavy calculations. * Preserves an order of data. */ -class ForkProcessor : public IProcessor +class ForkProcessor final : public IProcessor { public: ForkProcessor(const Block & header, size_t num_outputs) diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 8865eab732a..0d8c5f4ea47 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -15,7 +15,7 @@ namespace DB /// /// always_read_till_end - read all data from input ports even if limit was reached. /// with_ties, description - implementation of LIMIT WITH TIES. It works only for single port. -class LimitTransform : public IProcessor +class LimitTransform final : public IProcessor { private: UInt64 limit; diff --git a/src/Processors/OffsetTransform.h b/src/Processors/OffsetTransform.h index e67685362aa..d24440d68ea 100644 --- a/src/Processors/OffsetTransform.h +++ b/src/Processors/OffsetTransform.h @@ -10,7 +10,7 @@ namespace DB /// Implementation for OFFSET N (without limit) /// This processor support multiple inputs and outputs (the same number). /// Each pair of input and output port works independently. -class OffsetTransform : public IProcessor +class OffsetTransform final : public IProcessor { private: UInt64 offset; diff --git a/src/Processors/QueueBuffer.h b/src/Processors/QueueBuffer.h index 6856e214823..0736d6fbf43 100644 --- a/src/Processors/QueueBuffer.h +++ b/src/Processors/QueueBuffer.h @@ -10,7 +10,7 @@ namespace DB /** Reads all data into queue. * After all data has been read - output it in the same order. */ -class QueueBuffer : public IAccumulatingTransform +class QueueBuffer final : public IAccumulatingTransform { private: std::queue chunks; diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index 364d1b4c883..07d7149ebb4 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -18,7 +18,7 @@ namespace DB * - union data from multiple inputs to single output - to serialize data that was processed in parallel. * - split data from single input to multiple outputs - to allow further parallel processing. */ -class ResizeProcessor : public IProcessor +class ResizeProcessor final : public IProcessor { public: /// TODO Check that there is non zero number of inputs and outputs. From 780c8ea586be70a542325c170f9d2ffebed3d93f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 2 Feb 2023 19:29:03 +0100 Subject: [PATCH 43/78] Avoid leaving symbols leftovers on the screen during query fuzzy search In case of multi-line queries in the history, skim may leave some symbols on the screen, which looks icky. Signed-off-by: Azat Khuzhin --- rust/skim/src/lib.rs | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/rust/skim/src/lib.rs b/rust/skim/src/lib.rs index 90f39cc8382..00ee6c8c9c2 100644 --- a/rust/skim/src/lib.rs +++ b/rust/skim/src/lib.rs @@ -11,11 +11,26 @@ mod ffi { struct Item { text: String, + orig_text: String, +} +impl Item { + fn new(text: String) -> Self { + return Self{ + // Text that will be shown should not contains new lines since in this case skim may + // live some symbols on the screen, and this looks odd. + text: text.replace("\n", " "), + orig_text: text, + }; + } } impl SkimItem for Item { fn text(&self) -> Cow { return Cow::Borrowed(&self.text); } + + fn output(&self) -> Cow { + return Cow::Borrowed(&self.orig_text); + } } fn skim(prefix: &CxxString, words: &CxxVector) -> Result { @@ -34,7 +49,7 @@ fn skim(prefix: &CxxString, words: &CxxVector) -> Result Date: Thu, 2 Feb 2023 19:30:23 +0100 Subject: [PATCH 44/78] Avoid client prompt overlap after query fuzzy search Under certain conditions it is possible for skim to overlap the prompt, well, not overlap, but not re-render it, and so the client does not have a nice prompt. Signed-off-by: Azat Khuzhin --- src/Client/ReplxxLineReader.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 9fc0fec761d..405cb7af03b 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -417,6 +417,10 @@ ReplxxLineReader::ReplxxLineReader( { rx.print("skim failed: %s (consider using Ctrl-T for a regular non-fuzzy reverse search)\n", e.what()); } + + /// REPAINT before to avoid prompt overlap by the query + rx.invoke(Replxx::ACTION::REPAINT, code); + if (!new_query.empty()) rx.set_state(replxx::Replxx::State(new_query.c_str(), static_cast(new_query.size()))); From bb009fe25d69898917f3a445e244d4f28ab73e23 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 20:10:59 +0100 Subject: [PATCH 45/78] Update tests --- ..._log_queries_with_extensive_info.reference | 2 +- .../01702_system_query_log.reference | 60 +++++++++---------- .../02117_show_create_table_system.reference | 1 + 3 files changed, 32 insertions(+), 31 deletions(-) diff --git a/tests/queries/0_stateless/01600_log_queries_with_extensive_info.reference b/tests/queries/0_stateless/01600_log_queries_with_extensive_info.reference index 701e72b3b8e..bb6a99fa522 100644 --- a/tests/queries/0_stateless/01600_log_queries_with_extensive_info.reference +++ b/tests/queries/0_stateless/01600_log_queries_with_extensive_info.reference @@ -16,7 +16,7 @@ alter table test_log_queries.logtable rename column j to x, rename column k to y alter table test_log_queries.logtable2 add column x int, add column y int 1199561338572582360 Alter ['test_log_queries'] ['test_log_queries.logtable2'] ['test_log_queries.logtable2.x','test_log_queries.logtable2.y'] alter table test_log_queries.logtable3 drop column i, drop column k 340702370038862784 Alter ['test_log_queries'] ['test_log_queries.logtable3'] ['test_log_queries.logtable3.i','test_log_queries.logtable3.k'] rename table test_log_queries.logtable2 to test_log_queries.logtable4, test_log_queries.logtable3 to test_log_queries.logtable5 17256232154191063008 Rename ['test_log_queries'] ['test_log_queries.logtable2','test_log_queries.logtable3','test_log_queries.logtable4','test_log_queries.logtable5'] [] -optimize table test_log_queries.logtable 12932884188099170316 ['test_log_queries'] ['test_log_queries.logtable'] [] +optimize table test_log_queries.logtable 12932884188099170316 Optimize ['test_log_queries'] ['test_log_queries.logtable'] [] drop table if exists test_log_queries.logtable 9614905142075064664 Drop ['test_log_queries'] ['test_log_queries.logtable'] [] drop table if exists test_log_queries.logtable2 5276868561533661466 Drop ['test_log_queries'] ['test_log_queries.logtable2'] [] drop table if exists test_log_queries.logtable3 4776768361842582387 Drop ['test_log_queries'] ['test_log_queries.logtable3'] [] diff --git a/tests/queries/0_stateless/01702_system_query_log.reference b/tests/queries/0_stateless/01702_system_query_log.reference index 4b9eeb139f4..c653021aa5a 100644 --- a/tests/queries/0_stateless/01702_system_query_log.reference +++ b/tests/queries/0_stateless/01702_system_query_log.reference @@ -10,27 +10,27 @@ Misc queries ACTUAL LOG CONTENT: Select SELECT \'DROP queries and also a cleanup before the test\'; Drop DROP DATABASE IF EXISTS sqllt SYNC; - DROP USER IF EXISTS sqllt_user; - DROP ROLE IF EXISTS sqllt_role; - DROP POLICY IF EXISTS sqllt_policy ON sqllt.table, sqllt.view, sqllt.dictionary; - DROP ROW POLICY IF EXISTS sqllt_row_policy ON sqllt.table, sqllt.view, sqllt.dictionary; - DROP QUOTA IF EXISTS sqllt_quota; - DROP SETTINGS PROFILE IF EXISTS sqllt_settings_profile; +Drop DROP USER IF EXISTS sqllt_user; +Drop DROP ROLE IF EXISTS sqllt_role; +Drop DROP POLICY IF EXISTS sqllt_policy ON sqllt.table, sqllt.view, sqllt.dictionary; +Drop DROP ROW POLICY IF EXISTS sqllt_row_policy ON sqllt.table, sqllt.view, sqllt.dictionary; +Drop DROP QUOTA IF EXISTS sqllt_quota; +Drop DROP SETTINGS PROFILE IF EXISTS sqllt_settings_profile; Select SELECT \'CREATE queries\'; Create CREATE DATABASE sqllt; Create CREATE TABLE sqllt.table\n(\n i UInt8, s String\n)\nENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); Create CREATE VIEW sqllt.view AS SELECT i, s FROM sqllt.table; Create CREATE DICTIONARY sqllt.dictionary (key UInt64, value UInt64) PRIMARY KEY key SOURCE(CLICKHOUSE(DB \'sqllt\' TABLE \'table\' HOST \'localhost\' PORT 9001)) LIFETIME(0) LAYOUT(FLAT()); - CREATE USER sqllt_user IDENTIFIED WITH plaintext_password - CREATE ROLE sqllt_role; - CREATE POLICY sqllt_policy ON sqllt.table, sqllt.view, sqllt.dictionary AS PERMISSIVE TO ALL; - CREATE POLICY sqllt_row_policy ON sqllt.table, sqllt.view, sqllt.dictionary AS PERMISSIVE TO ALL; - CREATE QUOTA sqllt_quota KEYED BY user_name TO sqllt_role; - CREATE SETTINGS PROFILE sqllt_settings_profile SETTINGS interactive_delay = 200000; +Create CREATE USER sqllt_user IDENTIFIED WITH plaintext_password +Create CREATE ROLE sqllt_role; +Create CREATE POLICY sqllt_policy ON sqllt.table, sqllt.view, sqllt.dictionary AS PERMISSIVE TO ALL; +Create CREATE POLICY sqllt_row_policy ON sqllt.table, sqllt.view, sqllt.dictionary AS PERMISSIVE TO ALL; +Create CREATE QUOTA sqllt_quota KEYED BY user_name TO sqllt_role; +Create CREATE SETTINGS PROFILE sqllt_settings_profile SETTINGS interactive_delay = 200000; Grant GRANT sqllt_role TO sqllt_user; Select SELECT \'SET queries\'; - SET log_profile_events=false; - SET DEFAULT ROLE sqllt_role TO sqllt_user; +Set SET log_profile_events=false; +Set SET DEFAULT ROLE sqllt_role TO sqllt_user; Select -- SET ROLE sqllt_role; -- tests are executed by user `default` which is defined in XML and is impossible to update.\n\nSELECT \'ALTER TABLE queries\'; Alter ALTER TABLE sqllt.table ADD COLUMN new_col UInt32 DEFAULT 123456789; Alter ALTER TABLE sqllt.table COMMENT COLUMN new_col \'dummy column with a comment\'; @@ -54,19 +54,19 @@ System SYSTEM START FETCHES sqllt.table System SYSTEM STOP REPLICATED SENDS sqllt.table System SYSTEM START REPLICATED SENDS sqllt.table Select -- SYSTEM RELOAD DICTIONARY sqllt.dictionary; -- temporary out of order: Code: 210, Connection refused (localhost:9001) (version 21.3.1.1)\n-- DROP REPLICA\n-- haha, no\n-- SYSTEM KILL;\n-- SYSTEM SHUTDOWN;\n\n-- Since we don\'t really care about the actual output, suppress it with `FORMAT Null`.\nSELECT \'SHOW queries\'; - SHOW CREATE TABLE sqllt.table FORMAT Null; - SHOW CREATE DICTIONARY sqllt.dictionary FORMAT Null; - SHOW DATABASES LIKE \'sqllt\' FORMAT Null; - SHOW TABLES FROM sqllt FORMAT Null; - SHOW DICTIONARIES FROM sqllt FORMAT Null; - SHOW GRANTS FORMAT Null; - SHOW GRANTS FOR sqllt_user FORMAT Null; - SHOW CREATE USER sqllt_user FORMAT Null; - SHOW CREATE ROLE sqllt_role FORMAT Null; - SHOW CREATE POLICY sqllt_policy FORMAT Null; - SHOW CREATE ROW POLICY sqllt_row_policy FORMAT Null; - SHOW CREATE QUOTA sqllt_quota FORMAT Null; - SHOW CREATE SETTINGS PROFILE sqllt_settings_profile FORMAT Null; +Show SHOW CREATE TABLE sqllt.table FORMAT Null; +Show SHOW CREATE DICTIONARY sqllt.dictionary FORMAT Null; +Show SHOW DATABASES LIKE \'sqllt\' FORMAT Null; +Show SHOW TABLES FROM sqllt FORMAT Null; +Show SHOW DICTIONARIES FROM sqllt FORMAT Null; +Show SHOW GRANTS FORMAT Null; +Show SHOW GRANTS FOR sqllt_user FORMAT Null; +Show SHOW CREATE USER sqllt_user FORMAT Null; +Show SHOW CREATE ROLE sqllt_role FORMAT Null; +Show SHOW CREATE POLICY sqllt_policy FORMAT Null; +Show SHOW CREATE ROW POLICY sqllt_row_policy FORMAT Null; +Show SHOW CREATE QUOTA sqllt_quota FORMAT Null; +Show SHOW CREATE SETTINGS PROFILE sqllt_settings_profile FORMAT Null; Select SELECT \'GRANT queries\'; Grant GRANT SELECT ON sqllt.table TO sqllt_user; Grant GRANT DROP ON sqllt.view TO sqllt_user; @@ -74,13 +74,13 @@ Select SELECT \'REVOKE queries\'; Revoke REVOKE SELECT ON sqllt.table FROM sqllt_user; Revoke REVOKE DROP ON sqllt.view FROM sqllt_user; Select SELECT \'Misc queries\'; - DESCRIBE TABLE sqllt.table FORMAT Null; - CHECK TABLE sqllt.table FORMAT Null; +Describe DESCRIBE TABLE sqllt.table FORMAT Null; +Check CHECK TABLE sqllt.table FORMAT Null; Drop DETACH TABLE sqllt.table; Create ATTACH TABLE sqllt.table; Rename RENAME TABLE sqllt.table TO sqllt.table_new; Rename RENAME TABLE sqllt.table_new TO sqllt.table; Drop TRUNCATE TABLE sqllt.table; Drop DROP TABLE sqllt.table SYNC; - SET log_comment=\'\'; +Set SET log_comment=\'\'; DROP queries and also a cleanup after the test diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 80fe6aeefec..4381e3d7dee 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -612,6 +612,7 @@ CREATE TABLE system.processes `memory_usage` Int64, `peak_memory_usage` Int64, `query` String, + `query_kind` String, `thread_ids` Array(UInt64), `ProfileEvents` Map(String, UInt64), `Settings` Map(String, String), From b72810a608ca90174f081b8f8a074e388dfe627d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 Feb 2023 20:16:53 +0100 Subject: [PATCH 46/78] Fix tests --- tests/integration/test_storage_rabbitmq/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 31799e3a7aa..72b2bb39a00 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1057,6 +1057,8 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): """ ) + instance.wait_for_log_line("Started streaming to 1 attached views") + messages_num = 100000 def insert(): @@ -2982,7 +2984,9 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): insert_messages = [] def onReceived(channel, method, properties, body): - insert_messages.append(body.decode()) + message = body.decode() + insert_messages.append(message) + print(f"Received {len(insert_messages)} message: {message}") if len(insert_messages) == 2: channel.stop_consuming() From ebbc22bcf1a30d18b049ee18c09e7366a1cbec49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 Feb 2023 02:16:30 +0100 Subject: [PATCH 47/78] Fix error --- src/Interpreters/ProcessList.h | 4 ++-- src/Interpreters/QueryLog.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 6756224addc..d5c136ab62a 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -52,7 +52,7 @@ class ProcessListEntry; struct QueryStatusInfo { String query; - IAST::QueryKind query_kind; + IAST::QueryKind query_kind{}; UInt64 elapsed_microseconds; size_t read_rows; size_t read_bytes; @@ -136,7 +136,7 @@ protected: OvercommitTracker * global_overcommit_tracker = nullptr; /// This is used to control the maximum number of SELECT or INSERT queries. - IAST::QueryKind query_kind; + IAST::QueryKind query_kind{}; /// This field is unused in this class, but it /// increments/decrements metric in constructor/destructor. diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 00bdf9df30c..cb9cf289e7a 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -59,7 +59,7 @@ struct QueryLogElement String formatted_query; UInt64 normalized_query_hash{}; - IAST::QueryKind query_kind; + IAST::QueryKind query_kind{}; std::set query_databases; std::set query_tables; std::set query_columns; From 8ca602a148be76f6a1df066fddf693b98800ba76 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Feb 2023 02:58:28 +0000 Subject: [PATCH 48/78] check dynamic columns of part before its commit --- src/Storages/MergeTree/MergeTreeData.cpp | 24 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 1 + ...02553_type_json_attach_partition.reference | 1 + .../02553_type_json_attach_partition.sql | 14 +++++++++++ 4 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/02553_type_json_attach_partition.reference create mode 100644 tests/queries/0_stateless/02553_type_json_attach_partition.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b2e0c14489a..d1b7e2eaf5b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3393,6 +3393,28 @@ void MergeTreeData::checkPartDuplicate(MutableDataPartPtr & part, Transaction & } } +void MergeTreeData::checkPartDynamicColumns(MutableDataPartPtr & part, DataPartsLock & /*lock*/) const +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + const auto & columns = metadata_snapshot->getColumns(); + + if (!hasDynamicSubcolumns(columns)) + return; + + const auto & part_columns = part->getColumns(); + for (const auto & part_column : part_columns) + { + auto storage_column = columns.getPhysical(part_column.name); + if (!storage_column.type->hasDynamicSubcolumns()) + continue; + + auto concrete_storage_column = object_columns.getPhysical(part_column.name); + + /// It will throw if types are incompatible. + getLeastCommonTypeForDynamicColumns(storage_column.type, {concrete_storage_column.type, part_column.type}, true); + } +} + void MergeTreeData::preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction, bool need_rename) { part->is_temp = false; @@ -3427,6 +3449,7 @@ bool MergeTreeData::addTempPart( checkPartPartition(part, lock); checkPartDuplicate(part, out_transaction, lock); + checkPartDynamicColumns(part, lock); DataPartPtr covering_part; DataPartsVector covered_parts = getActivePartsToReplace(part->info, part->name, covering_part, lock); @@ -3467,6 +3490,7 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( part->assertState({DataPartState::Temporary}); checkPartPartition(part, lock); checkPartDuplicate(part, out_transaction, lock); + checkPartDynamicColumns(part, lock); PartHierarchy hierarchy = getPartHierarchy(part->info, DataPartState::Active, lock); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7dcd0c40553..b9531e4020b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1417,6 +1417,7 @@ private: /// Checking that candidate part doesn't break invariants: correct partition void checkPartPartition(MutableDataPartPtr & part, DataPartsLock & lock) const; void checkPartDuplicate(MutableDataPartPtr & part, Transaction & transaction, DataPartsLock & lock) const; + void checkPartDynamicColumns(MutableDataPartPtr & part, DataPartsLock & lock) const; /// Preparing itself to be committed in memory: fill some fields inside part, add it to data_parts_indexes /// in precommitted state and to transaction diff --git a/tests/queries/0_stateless/02553_type_json_attach_partition.reference b/tests/queries/0_stateless/02553_type_json_attach_partition.reference new file mode 100644 index 00000000000..611090b5207 --- /dev/null +++ b/tests/queries/0_stateless/02553_type_json_attach_partition.reference @@ -0,0 +1 @@ +{"b":"1","c":{"k1":[1,2]}} diff --git a/tests/queries/0_stateless/02553_type_json_attach_partition.sql b/tests/queries/0_stateless/02553_type_json_attach_partition.sql new file mode 100644 index 00000000000..9225106f767 --- /dev/null +++ b/tests/queries/0_stateless/02553_type_json_attach_partition.sql @@ -0,0 +1,14 @@ +SET allow_experimental_object_type = 1; + +DROP TABLE IF EXISTS t_json_attach_partition; + +CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}}; +ALTER TABLE t_json_attach_partition DETACH PARTITION tuple(); +INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": [1, 2]}}; + +ALTER TABLE t_json_attach_partition ATTACH PARTITION tuple(); -- { serverError TYPE_MISMATCH } +SELECT * FROM t_json_attach_partition FORMAT JSONEachRow; + +DROP TABLE t_json_attach_partition; From 93ead69b7f96ab03222b284d8d89bf0c9bc3ad10 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 11:05:54 +0800 Subject: [PATCH 49/78] Resolve review comments --- src/Common/OpenTelemetryTraceContext.cpp | 41 +++++++++++++++--------- src/Common/OpenTelemetryTraceContext.h | 8 +++-- 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 65e66b1da96..22fe4ae3f21 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -19,7 +19,7 @@ bool Span::addAttribute(std::string_view name, UInt64 value) noexcept if (!this->isTraceEnabled() || name.empty()) return false; - return addAttribute(name, toString(value)); + return addAttributeImpl(name, toString(value)); } bool Span::addAttributeIfNotZero(std::string_view name, UInt64 value) noexcept @@ -27,7 +27,7 @@ bool Span::addAttributeIfNotZero(std::string_view name, UInt64 value) noexcept if (!this->isTraceEnabled() || name.empty() || value == 0) return false; - return addAttribute(name, toString(value)); + return addAttributeImpl(name, toString(value)); } bool Span::addAttribute(std::string_view name, std::string_view value) noexcept @@ -35,30 +35,26 @@ bool Span::addAttribute(std::string_view name, std::string_view value) noexcept if (!this->isTraceEnabled() || name.empty()) return false; - try - { - this->attributes.push_back(Tuple{name, value}); - } - catch (...) - { - return false; - } - return true; + return addAttributeImpl(name, value); } bool Span::addAttributeIfNotEmpty(std::string_view name, std::string_view value) noexcept { - return value.empty() ? false : addAttribute(name, value); + if (!this->isTraceEnabled() || name.empty() || value.empty()) + return false; + + return addAttributeImpl(name, value); } bool Span::addAttribute(std::string_view name, std::function value_supplier) noexcept { - if (!this->isTraceEnabled() || !value_supplier) + if (!this->isTraceEnabled() || name.empty() || !value_supplier) return false; try { - return addAttributeIfNotEmpty(name, value_supplier()); + const String value = value_supplier(); + return value.empty() ? false : addAttributeImpl(name, value); } catch (...) { @@ -72,7 +68,7 @@ bool Span::addAttribute(const Exception & e) noexcept if (!this->isTraceEnabled()) return false; - return addAttribute("clickhouse.exception", getExceptionMessage(e, false)); + return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false)); } bool Span::addAttribute(std::exception_ptr e) noexcept @@ -80,7 +76,20 @@ bool Span::addAttribute(std::exception_ptr e) noexcept if (!this->isTraceEnabled() || e == nullptr) return false; - return addAttribute("clickhouse.exception", getExceptionMessage(e, false)); + return addAttributeImpl("clickhouse.exception", getExceptionMessage(e, false)); +} + +bool Span::addAttributeImpl(std::string_view name, std::string_view value) noexcept +{ + try + { + this->attributes.push_back(Tuple{name, value}); + } + catch (...) + { + return false; + } + return true; } SpanHolder::SpanHolder(std::string_view _operation_name) diff --git a/src/Common/OpenTelemetryTraceContext.h b/src/Common/OpenTelemetryTraceContext.h index 930262f2c92..64bce9e98db 100644 --- a/src/Common/OpenTelemetryTraceContext.h +++ b/src/Common/OpenTelemetryTraceContext.h @@ -23,8 +23,9 @@ struct Span UInt64 finish_time_us = 0; Map attributes; - /// Following methods are declared as noexcept to make sure they're exception safe - /// This is because sometimes they will be called in exception handlers/dtor + /// Following methods are declared as noexcept to make sure they're exception safe. + /// This is because sometimes they will be called in exception handlers/dtor. + /// Returns true if attribute is successfully added and false otherwise. bool addAttribute(std::string_view name, UInt64 value) noexcept; bool addAttributeIfNotZero(std::string_view name, UInt64 value) noexcept; bool addAttribute(std::string_view name, std::string_view value) noexcept; @@ -37,6 +38,9 @@ struct Span { return trace_id != UUID(); } + +private: + bool addAttributeImpl(std::string_view name, std::string_view value) noexcept; }; /// See https://www.w3.org/TR/trace-context/ for trace_flags definition From 7ad4b3176a086e31b8a450be98cdc9569505e2a4 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 11:13:45 +0800 Subject: [PATCH 50/78] Clean code --- src/Common/OpenTelemetryTraceContext.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index 22fe4ae3f21..37455f2266d 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -53,7 +53,7 @@ bool Span::addAttribute(std::string_view name, std::function value_sup try { - const String value = value_supplier(); + auto value = value_supplier(); return value.empty() ? false : addAttributeImpl(name, value); } catch (...) From ad09721238e48d064d8ffda13193d41e42f0bb21 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 11:59:17 +0800 Subject: [PATCH 51/78] Fix test case --- .../queries/0_stateless/02117_show_create_table_system.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 9e065c455e9..f7d0e6f7b5c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -920,6 +920,7 @@ CREATE TABLE system.replication_queue `is_currently_executing` UInt8, `num_tries` UInt32, `last_exception` String, + `last_exception_time` DateTime, `last_attempt_time` DateTime, `num_postponed` UInt32, `postpone_reason` String, From 4cf711f2d853c94e328ea69e53aa221bd086005c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 02:46:30 +0100 Subject: [PATCH 52/78] Fix IO URing --- tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh | 4 ++-- tests/queries/0_stateless/01632_tinylog_read_write.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh index b659d550fa4..69082f424b4 100755 --- a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh @@ -33,7 +33,7 @@ function thread_rename { function thread_select { while true; do - $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' sleep 0.0$RANDOM done } @@ -47,7 +47,7 @@ function thread_insert { function thread_insert_select { while true; do - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' sleep 0.0$RANDOM done } diff --git a/tests/queries/0_stateless/01632_tinylog_read_write.sh b/tests/queries/0_stateless/01632_tinylog_read_write.sh index e45fdd91ff6..69f985a9d0d 100755 --- a/tests/queries/0_stateless/01632_tinylog_read_write.sh +++ b/tests/queries/0_stateless/01632_tinylog_read_write.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test; CREATE TABLE function thread_select { while true; do - $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "SELECT * FROM test FORMAT Null" sleep 0.0$RANDOM done } From 700503d64afe26edd5ff1bca6cc298d2b63ef487 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Feb 2023 02:49:04 +0100 Subject: [PATCH 53/78] Fix IO URing --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1fa5384f005..2f7de4b5763 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -603,7 +603,7 @@ class IColumn; M(ShortCircuitFunctionEvaluation, short_circuit_function_evaluation, ShortCircuitFunctionEvaluation::ENABLE, "Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.", 0) \ \ M(LocalFSReadMethod, storage_file_read_method, LocalFSReadMethod::mmap, "Method of reading data from storage file, one of: read, pread, mmap.", 0) \ - M(String, local_filesystem_read_method, "pread_threadpool", "Method of reading data from local filesystem, one of: read, pread, mmap, io_uring, pread_threadpool.", 0) \ + M(String, local_filesystem_read_method, "pread_threadpool", "Method of reading data from local filesystem, one of: read, pread, mmap, io_uring, pread_threadpool. The 'io_uring' method is experimental and does not work for Log, TinyLog, StripeLog, File, Set and Join, and other tables with append-able files in presence of concurrent reads and writes.", 0) \ M(String, remote_filesystem_read_method, "threadpool", "Method of reading data from remote filesystem, one of: read, threadpool.", 0) \ M(Bool, local_filesystem_read_prefetch, false, "Should use prefetching when reading data from local filesystem.", 0) \ M(Bool, remote_filesystem_read_prefetch, true, "Should use prefetching when reading data from remote filesystem.", 0) \ From a83a81fe3beebe36e227ba70a22b731b3524476a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 Feb 2023 07:05:02 +0100 Subject: [PATCH 54/78] Fix tests --- .../01502_log_tinylog_deadlock_race.reference | 2 - .../01502_log_tinylog_deadlock_race.sh | 91 ------------------- .../01502_long_log_tinylog_deadlock_race.sh | 4 +- 3 files changed, 2 insertions(+), 95 deletions(-) delete mode 100644 tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference delete mode 100755 tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference deleted file mode 100644 index c62a2b18918..00000000000 --- a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference +++ /dev/null @@ -1,2 +0,0 @@ -Testing Memory -Done Memory diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh deleted file mode 100755 index 69082f424b4..00000000000 --- a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh +++ /dev/null @@ -1,91 +0,0 @@ -#!/usr/bin/env bash -# Tags: deadlock - -set -e - -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -function thread_create { - while true; do - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" - sleep 0.0$RANDOM - done -} - -function thread_drop { - while true; do - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" - sleep 0.0$RANDOM - done -} - -function thread_rename { - while true; do - $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM - done -} - -function thread_select { - while true; do - $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -function thread_insert { - while true; do - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: '| grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -function thread_insert_select { - while true; do - $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -export -f thread_create -export -f thread_drop -export -f thread_rename -export -f thread_select -export -f thread_insert -export -f thread_insert_select - - -# Do randomized queries and expect nothing extraordinary happens. - -function test_with_engine { - echo "Testing $1" - - timeout 10 bash -c "thread_create t1 $1" & - timeout 10 bash -c "thread_create t2 $1" & - timeout 10 bash -c 'thread_drop t1' & - timeout 10 bash -c 'thread_drop t2' & - timeout 10 bash -c 'thread_rename t1 t2' & - timeout 10 bash -c 'thread_rename t2 t1' & - timeout 10 bash -c 'thread_select t1' & - timeout 10 bash -c 'thread_select t2' & - timeout 10 bash -c 'thread_insert t1 5' & - timeout 10 bash -c 'thread_insert t2 10' & - timeout 10 bash -c 'thread_insert_select t1 t2' & - timeout 10 bash -c 'thread_insert_select t2 t1' & - - wait - echo "Done $1" -} - -#test_with_engine TinyLog -#test_with_engine StripeLog -#test_with_engine Log -test_with_engine Memory - -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t2" diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh index 3cf94a0b2bd..b8efee89b4a 100755 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -33,7 +33,7 @@ function thread_rename { function thread_select { while true; do - $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' sleep 0.0$RANDOM done } @@ -47,7 +47,7 @@ function thread_insert { function thread_insert_select { while true; do - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + $CLICKHOUSE_CLIENT --local_filesystem_read_method pread --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' sleep 0.0$RANDOM done } From d38adfab307eba3ce57cdb68fd94d6c49914d365 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 14:41:27 +0800 Subject: [PATCH 55/78] Merge two overridden functions as one --- src/Common/Exception.cpp | 5 ----- src/Common/Exception.h | 3 +-- src/Interpreters/DDLWorker.cpp | 2 +- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 12 files changed, 18 insertions(+), 24 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index fa118c2fcf5..7e7ccfa4877 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -559,11 +559,6 @@ bool ExecutionStatus::tryDeserializeText(const std::string & data) return true; } -ExecutionStatus ExecutionStatus::fromCurrentException(bool with_stacktrace) -{ - return ExecutionStatus(getCurrentExceptionCode(), getCurrentExceptionMessage(with_stacktrace, true)); -} - ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message, bool with_stacktrace) { String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(with_stacktrace, true); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 73524a0202c..8e50c1114f4 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -242,8 +242,7 @@ struct ExecutionStatus explicit ExecutionStatus(int return_code, const std::string & exception_message = "") : code(return_code), message(exception_message) {} - static ExecutionStatus fromCurrentException(bool with_stacktrace = false); - static ExecutionStatus fromCurrentException(const std::string & start_of_message, bool with_stacktrace = false); + static ExecutionStatus fromCurrentException(const std::string & start_of_message = "", bool with_stacktrace = false); static ExecutionStatus fromText(const std::string & data); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0f91212e6a9..c8673bed842 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -632,7 +632,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (task.is_initial_query) throw; tryLogCurrentException(log, "An error occurred before execution of DDL task: "); - task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution"); + task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution", false); } if (task.execution_status.code != 0) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index be3074c40d2..2d2013bd648 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -335,7 +335,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite " We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(part), "merge-not-byte-identical"); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index e59f7fda44c..9f24839f1e1 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -52,7 +52,7 @@ bool MergePlainMergeTreeTask::executeStep() } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83203542cd3..477d75474b3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7469,7 +7469,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (cloned_part) cloned_part->remove(); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index e785a38d501..b83c058f7fd 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -215,7 +215,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. " "We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); if (storage.getSettings()->detach_not_byte_identical_parts) storage.forcefullyMovePartToDetachedAndRemoveFromMemory(std::move(new_part), "mutate-not-byte-identical"); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index adc5b0c2c22..76ba921b705 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -103,7 +103,7 @@ bool MutatePlainMergeTreeTask::executeStep() PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message.text); - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); tryLogCurrentException(__PRETTY_FUNCTION__); return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 376c3e46a22..a4a4de6d8d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -188,7 +188,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException(true)); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -204,7 +204,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() catch (...) { if (part_log_writer) - part_log_writer(ExecutionStatus::fromCurrentException(true)); + part_log_writer(ExecutionStatus::fromCurrentException("", true)); throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index fb8affa5ee7..ee192966758 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -520,7 +520,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(true)); + PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -588,7 +588,7 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1ee9f0e9ae0..9d6ddc2c79a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1849,7 +1849,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -1932,7 +1932,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0641d3ad8f2..ff3c039be1d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2337,7 +2337,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -4190,7 +4190,7 @@ bool StorageReplicatedMergeTree::fetchPart( catch (...) { if (!to_detached) - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -4300,7 +4300,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( } catch (...) { - write_part_log(ExecutionStatus::fromCurrentException(true)); + write_part_log(ExecutionStatus::fromCurrentException("", true)); throw; } @@ -7041,7 +7041,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7270,7 +7270,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException(true)); + PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 7831cf85db5fccf18e5817efb5120124e60d53d2 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Feb 2023 15:36:12 +0800 Subject: [PATCH 56/78] Revert an extra change --- src/Interpreters/DDLWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index c8673bed842..0f91212e6a9 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -632,7 +632,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (task.is_initial_query) throw; tryLogCurrentException(log, "An error occurred before execution of DDL task: "); - task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution", false); + task.execution_status = ExecutionStatus::fromCurrentException("An error occurred before execution"); } if (task.execution_status.code != 0) From b95ecfaec549031675769738c6f93568e82c7628 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 3 Feb 2023 10:00:32 +0000 Subject: [PATCH 57/78] Minor doc fixes for inverted index --- .../mergetree-family/invertedindexes.md | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 9f759303034..aa11258dc4a 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -86,9 +86,9 @@ CREATE TABLE hackernews ( author String, timestamp DateTime, comment String, - dead UInt8, - parent UInt64, - poll UInt64, + dead UInt8, + parent UInt64, + poll UInt64, children Array(UInt32), url String, score UInt32, @@ -171,9 +171,23 @@ WHERE hasToken(lower(comment), 'clickhouse') 1 row in set. Elapsed: 0.747 sec. Processed 4.49 million rows, 1.77 GB (6.01 million rows/s., 2.37 GB/s.) ``` +We can also search for one or all of multiple terms, i.e., disjunctions or conjunctions: + +```sql +-- multiple OR'ed terms +SELECT count(*) +FROM hackernews +WHERE multiSearchAny(lower(comment), ['oltp', 'olap']); + +-- multiple AND'ed terms +SELECT count(*) +FROM hackernews +WHERE hasToken(lower(comment), 'avx') AND hasToken(lower(comment), 'sve'); +``` + :::note Unlike other secondary indices, inverted indexes (for now) map to row numbers (row ids) instead of granule ids. The reason for this design is performance. In practice, users often search for multiple terms at once. For example, filter predicate `WHERE s LIKE '%little%' OR s LIKE '%big%'` can be evaluated directly using an inverted index by forming the union of the row id lists for terms "little" and "big". This also means that the parameter `GRANULARITY` supplied to index creation has no meaning (it may be removed from the syntax in the future). -::: \ No newline at end of file +::: From cb0e338d70dfa412ab07f44ba1fdbb8fa8e0cca3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 Feb 2023 12:10:39 +0100 Subject: [PATCH 58/78] Revert "Improve behaviour of conversion into Date for boundary value 65535" --- src/Functions/FunctionsConversion.h | 4 +- ...lumn_consistent_insert_behaviour.reference | 27 ------- ...ate_column_consistent_insert_behaviour.sql | 72 ------------------- 3 files changed, 2 insertions(+), 101 deletions(-) delete mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference delete mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 55003044ff5..c34cd3ac875 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -377,7 +377,7 @@ struct ToDateTransform32Or64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // since converting to Date, no need in values outside of default LUT range. - return (from <= DATE_LUT_MAX_DAY_NUM) + return (from < DATE_LUT_MAX_DAY_NUM) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } @@ -394,7 +394,7 @@ struct ToDateTransform32Or64Signed /// The function should be monotonic (better for query optimizations), so we saturate instead of overflow. if (from < 0) return 0; - return (from <= DATE_LUT_MAX_DAY_NUM) + return (from < DATE_LUT_MAX_DAY_NUM) ? static_cast(from) : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference deleted file mode 100644 index 276c1456af6..00000000000 --- a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference +++ /dev/null @@ -1,27 +0,0 @@ -2149-06-06 65535 -2149-06-06 toUInt16(65535) -2149-06-06 toInt32(65535) -2149-06-06 toUInt32(65535) -2149-06-06 toDate(65535) -2149-06-06 CAST(65535 as UInt16) -2149-06-06 CAST(65535 as Int32) -2149-06-06 CAST(65535 as UInt32) -2149-06-06 CAST(65535 as Date) -2149-06-05 65534 -2149-06-05 toUInt16(65534) -2149-06-05 toInt32(65534) -2149-06-05 toUInt32(65534) -2149-06-05 toDate(65534) -2149-06-05 CAST(65534 as UInt16) -2149-06-05 CAST(65534 as Int32) -2149-06-05 CAST(65534 as UInt32) -2149-06-05 CAST(65534 as Date) -1970-01-01 65536 -1970-01-01 toUInt16(65536) -1970-01-01 toInt32(65536) -1970-01-01 toUInt32(65536) -1970-01-01 toDate(65536) -1970-01-01 CAST(65536 as UInt16) -1970-01-01 CAST(65536 as Int32) -1970-01-01 CAST(65536 as UInt32) -1970-01-01 CAST(65536 as Date) diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql deleted file mode 100644 index 20614f648ab..00000000000 --- a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql +++ /dev/null @@ -1,72 +0,0 @@ -DROP TABLE IF EXISTS 02540_date; -CREATE TABLE 02540_date (txt String, x Date) engine=Memory; - --- Date: Supported range of values: [1970-01-01, 2149-06-06]. --- ^----closed interval---^ - -INSERT INTO 02540_date VALUES('65535', 65535); -INSERT INTO 02540_date VALUES('toUInt16(65535)', toUInt16(65535)); -- #43370 weird one -> used to be 1970-01-01 -INSERT INTO 02540_date VALUES('toInt32(65535)', toInt32(65535)); -INSERT INTO 02540_date VALUES('toUInt32(65535)', toUInt32(65535)); -INSERT INTO 02540_date VALUES('toDate(65535)', toDate(65535)); - -INSERT INTO 02540_date VALUES('CAST(65535 as UInt16)', CAST(65535 as UInt16)); -INSERT INTO 02540_date VALUES('CAST(65535 as Int32)', CAST(65535 as Int32)); -INSERT INTO 02540_date VALUES('CAST(65535 as UInt32)', CAST(65535 as UInt32)); -INSERT INTO 02540_date VALUES('CAST(65535 as Date)', CAST(65535 as Date)); - -INSERT INTO 02540_date VALUES('65534', 65534); -INSERT INTO 02540_date VALUES('toUInt16(65534)', toUInt16(65534)); -INSERT INTO 02540_date VALUES('toInt32(65534)', toInt32(65534)); -INSERT INTO 02540_date VALUES('toUInt32(65534)', toUInt32(65534)); -INSERT INTO 02540_date VALUES('toDate(65534)', toDate(65534)); - -INSERT INTO 02540_date VALUES('CAST(65534 as UInt16)', CAST(65534 as UInt16)); -INSERT INTO 02540_date VALUES('CAST(65534 as Int32)', CAST(65534 as Int32)); -INSERT INTO 02540_date VALUES('CAST(65534 as UInt32)', CAST(65534 as UInt32)); -INSERT INTO 02540_date VALUES('CAST(65534 as Date)', CAST(65534 as Date)); - -INSERT INTO 02540_date VALUES('65536', 65536); -INSERT INTO 02540_date VALUES('toUInt16(65536)', toUInt16(65536)); -INSERT INTO 02540_date VALUES('toInt32(65536)', toInt32(65536)); -INSERT INTO 02540_date VALUES('toUInt32(65536)', toUInt32(65536)); -INSERT INTO 02540_date VALUES('toDate(65536)', toDate(65536)); - -INSERT INTO 02540_date VALUES('CAST(65536 as UInt16)', CAST(65536 as UInt16)); -INSERT INTO 02540_date VALUES('CAST(65536 as Int32)', CAST(65536 as Int32)); -INSERT INTO 02540_date VALUES('CAST(65536 as UInt32)', CAST(65536 as UInt32)); -INSERT INTO 02540_date VALUES('CAST(65536 as Date)', CAST(65536 as Date)); - - -SELECT x, txt FROM 02540_date WHERE txt == '65535'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65535)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65535)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65535)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65535)'; - -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt16)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Int32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Date)'; - -SELECT x, txt FROM 02540_date WHERE txt == '65534'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65534)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65534)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65534)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65534)'; - -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt16)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Int32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Date)'; - -SELECT x, txt FROM 02540_date WHERE txt == '65536'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65536)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65536)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65536)'; -SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65536)'; - -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt16)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as Int32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt32)'; -SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as Date)'; From 26a292dafcb4fff523c3f759fddec90b47004148 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 24 Jan 2023 14:46:59 +0100 Subject: [PATCH 59/78] Add systemd service-file for clickhouse-keeper --- packages/clickhouse-keeper.service | 27 +++++++++++++++++++++++++++ packages/clickhouse-keeper.yaml | 2 ++ 2 files changed, 29 insertions(+) create mode 100644 packages/clickhouse-keeper.service diff --git a/packages/clickhouse-keeper.service b/packages/clickhouse-keeper.service new file mode 100644 index 00000000000..2809074c93a --- /dev/null +++ b/packages/clickhouse-keeper.service @@ -0,0 +1,27 @@ +[Unit] +Description=ClickHouse Keeper - zookeeper compatible distributed coordination server +Requires=network-online.target +# NOTE: that After/Wants=time-sync.target is not enough, you need to ensure +# that the time was adjusted already, if you use systemd-timesyncd you are +# safe, but if you use ntp or some other daemon, you should configure it +# additionaly. +After=time-sync.target network-online.target +Wants=time-sync.target + +[Service] +Type=simple +User=clickhouse +Group=clickhouse +Restart=always +RestartSec=30 +RuntimeDirectory=%p # %p is resolved to the systemd unit name +ExecStart=/usr/bin/clickhouse-keeper --config=/etc/clickhouse-keeper/keeper_config.xml --pid-file=%t/%p/%p.pid +# Minus means that this file is optional. +EnvironmentFile=-/etc/default/%p +LimitCORE=infinity +LimitNOFILE=500000 +CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE + +[Install] +# ClickHouse should not start from the rescue shell (rescue.target). +WantedBy=multi-user.target diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml index 7586fa580e6..0706dc3fe7e 100644 --- a/packages/clickhouse-keeper.yaml +++ b/packages/clickhouse-keeper.yaml @@ -30,6 +30,8 @@ contents: type: config|noreplace - src: root/usr/bin/clickhouse-keeper dst: /usr/bin/clickhouse-keeper +- src: clickhouse-keeper.service + dst: /lib/systemd/system/clickhouse-keeper.service # docs - src: ../AUTHORS dst: /usr/share/doc/clickhouse-keeper/AUTHORS From 548b89b7e06be3f3d8382cc61cdad325442cba73 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 24 Jan 2023 14:48:28 +0100 Subject: [PATCH 60/78] Use systemd %-specifiers in clickhouse-server.service --- packages/clickhouse-server.service | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/packages/clickhouse-server.service b/packages/clickhouse-server.service index a1602482073..5ea30c062b0 100644 --- a/packages/clickhouse-server.service +++ b/packages/clickhouse-server.service @@ -17,10 +17,10 @@ User=clickhouse Group=clickhouse Restart=always RestartSec=30 -RuntimeDirectory=clickhouse-server -ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml --pid-file=/run/clickhouse-server/clickhouse-server.pid +RuntimeDirectory=%p # %p is resolved to the systemd unit name +ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml --pid-file=%t/%p/%p.pid # Minus means that this file is optional. -EnvironmentFile=-/etc/default/clickhouse +EnvironmentFile=-/etc/default/%p LimitCORE=infinity LimitNOFILE=500000 CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE From dc00bce9cf4551f0e6290d301f1e984f09d3aaf6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 25 Jan 2023 22:59:40 +0100 Subject: [PATCH 61/78] Add tests for packages' installation --- .github/workflows/pull_request.yml | 36 ++++ docker/images.json | 8 + docker/test/install/deb/Dockerfile | 64 +++++++ docker/test/install/rpm/Dockerfile | 55 ++++++ tests/ci/ci_config.py | 6 + tests/ci/install_check.py | 287 +++++++++++++++++++++++++++++ 6 files changed, 456 insertions(+) create mode 100644 docker/test/install/deb/Dockerfile create mode 100644 docker/test/install/rpm/Dockerfile create mode 100644 tests/ci/install_check.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c677ec4bf5c..be8c99adc43 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -984,6 +984,42 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" +############################################################################################ +#################################### INSTALL PACKAGES ###################################### +############################################################################################ + TestInstallPackagesRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/test_install + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Install packages (amd64) + REPO_COPY=${{runner.temp}}/test_install/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Test packages installation + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 install_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## diff --git a/docker/images.json b/docker/images.json index 323f53c0ae6..bd63aea24ba 100644 --- a/docker/images.json +++ b/docker/images.json @@ -134,6 +134,14 @@ "name": "clickhouse/keeper-jepsen-test", "dependent": [] }, + "docker/test/install/deb": { + "name": "clickhouse/install-deb-test", + "dependent": [] + }, + "docker/test/install/rpm": { + "name": "clickhouse/install-rpm-test", + "dependent": [] + }, "docker/docs/builder": { "name": "clickhouse/docs-builder", "dependent": [ diff --git a/docker/test/install/deb/Dockerfile b/docker/test/install/deb/Dockerfile new file mode 100644 index 00000000000..9614473c69b --- /dev/null +++ b/docker/test/install/deb/Dockerfile @@ -0,0 +1,64 @@ +FROM ubuntu:22.04 + +# The Dockerfile is nicely borrowed from +# https://github.com/lionelnicolas/docker-ubuntu-systemd/blob/83aa3249146f5df264fe45353f79fc76eb1e42d7/Dockerfile + +ENV \ + DEBIAN_FRONTEND=noninteractive \ + LANG=C.UTF-8 \ + container=docker \ + init=/lib/systemd/systemd + +# install systemd packages +RUN apt-get update && \ + apt-get install -y --no-install-recommends \ + systemd \ + && \ + apt-get clean && \ + rm -rf /var/lib/apt/lists + +# configure systemd +# remove systemd 'wants' triggers +# remove everything except tmpfiles setup in sysinit target +# remove UTMP updater service +# disable /tmp mount +# fix missing BPF firewall support warning +# just for cosmetics, fix "not-found" entries while using "systemctl --all" +RUN \ + find \ + /etc/systemd/system/*.wants/* \ + /lib/systemd/system/multi-user.target.wants/* \ + /lib/systemd/system/sockets.target.wants/*initctl* \ + ! -type d \ + -delete && \ + find \ + /lib/systemd/system/sysinit.target.wants \ + ! -type d \ + ! -name '*systemd-tmpfiles-setup*' \ + -delete && \ + find \ + /lib/systemd \ + -name systemd-update-utmp-runlevel.service \ + -delete && \ + rm -vf /usr/share/systemd/tmp.mount && \ + sed -ri '/^IPAddressDeny/d' /lib/systemd/system/systemd-journald.service && \ + for MATCH in \ + plymouth-start.service \ + plymouth-quit-wait.service \ + syslog.socket \ + syslog.service \ + display-manager.service \ + systemd-sysusers.service \ + tmp.mount \ + systemd-udevd.service \ + ; do \ + grep -rn --binary-files=without-match ${MATCH} /lib/systemd/ | cut -d: -f1 | xargs sed -ri 's/(.*=.*)'${MATCH}'(.*)/\1\2/'; \ + done && \ + systemctl disable ondemand.service && \ + systemctl set-default multi-user.target + +VOLUME ["/run", "/run/lock"] + +STOPSIGNAL SIGRTMIN+3 + +ENTRYPOINT ["/lib/systemd/systemd"] diff --git a/docker/test/install/rpm/Dockerfile b/docker/test/install/rpm/Dockerfile new file mode 100644 index 00000000000..c55e0fe2507 --- /dev/null +++ b/docker/test/install/rpm/Dockerfile @@ -0,0 +1,55 @@ +FROM centos:8 + +# The Dockerfile is nicely borrowed from +# https://github.com/lionelnicolas/docker-ubuntu-systemd/blob/83aa3249146f5df264fe45353f79fc76eb1e42d7/Dockerfile + +ENV \ + LANG=C.UTF-8 \ + container=docker \ + init=/lib/systemd/systemd + +# configure systemd +# remove systemd 'wants' triggers +# remove everything except tmpfiles setup in sysinit target +# remove UTMP updater service +# disable /tmp mount +# fix missing BPF firewall support warning +# just for cosmetics, fix "not-found" entries while using "systemctl --all" +RUN \ + find \ + /etc/systemd/system/*.wants/ \ + /lib/systemd/system/multi-user.target.wants/ \ + /lib/systemd/system/local-fs.target.wants/ \ + /lib/systemd/system/sockets.target.wants/*initctl* \ + ! -type d \ + -delete && \ + find \ + /lib/systemd/system/sysinit.target.wants \ + ! -type d \ + ! -name '*systemd-tmpfiles-setup*' \ + -delete && \ + find \ + /lib/systemd \ + -name systemd-update-utmp-runlevel.service \ + -delete && \ + rm -vf /usr/share/systemd/tmp.mount && \ + sed -ri '/^IPAddressDeny/d' /lib/systemd/system/systemd-journald.service && \ + for MATCH in \ + plymouth-start.service \ + plymouth-quit-wait.service \ + syslog.socket \ + syslog.service \ + display-manager.service \ + systemd-sysusers.service \ + tmp.mount \ + systemd-udevd.service \ + ; do \ + grep -rn --binary-files=without-match ${MATCH} /lib/systemd/ | cut -d: -f1 | xargs sed -ri 's/(.*=.*)'${MATCH}'(.*)/\1\2/'; \ + done && \ + systemctl set-default multi-user.target + +VOLUME ["/run", "/run/lock"] + +STOPSIGNAL SIGRTMIN+3 + +ENTRYPOINT ["/lib/systemd/systemd"] diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c77acfb679f..6aa29465c17 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -182,6 +182,12 @@ CI_CONFIG = { "tests_config": { # required_build - build name for artifacts # force_tests - force success status for tests + "Install packages (amd64)": { + "required_build": "package_release", + }, + "Install packages (arm64)": { + "required_build": "package_aarch64", + }, "Stateful tests (asan)": { "required_build": "package_asan", }, diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py new file mode 100644 index 00000000000..e45a4fffe96 --- /dev/null +++ b/tests/ci/install_check.py @@ -0,0 +1,287 @@ +#!/usr/bin/env python3 + +import argparse + +import atexit +import logging +import sys +import subprocess +from pathlib import Path + +from typing import Dict + +from github import Github + +from build_download_helper import download_builds_filter +from clickhouse_helper import ( + ClickHouseHelper, + mark_flaky_tests, + prepare_tests_results_for_clickhouse, +) +from commit_status_helper import post_commit_status, update_mergeable_check +from docker_pull_helper import get_image_with_version, DockerImage +from env_helper import TEMP_PATH as TEMP, REPORTS_PATH +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from report import TestResults, TestResult +from rerun_helper import RerunHelper +from s3_helper import S3Helper +from stopwatch import Stopwatch +from tee_popen import TeePopen +from upload_result_helper import upload_results + + +RPM_IMAGE = "clickhouse/install-rpm-test" +DEB_IMAGE = "clickhouse/install-deb-test" +TEMP_PATH = Path(TEMP) +SUCCESS = "success" +FAILURE = "failure" + + +def prepare_test_scripts(): + server_test = """#!/bin/bash +systemctl start clickhouse-server +clickhouse-client -q 'SELECT version()'""" + keeper_test = """#!/bin/bash +systemctl start clickhouse-keeper +for i in {1..20}; do + echo wait for clickhouse-keeper to being up + > /dev/tcp/127.0.0.1/9181 2>/dev/null && break || sleep 1 +done +for i in {1..5}; do + echo wait for clickhouse-keeper to answer on mntr request + exec 13<>/dev/tcp/127.0.0.1/9181 + echo mntr >&13 + cat <&13 | grep zk_version && break || sleep 1 + exec 13>&- +done +exec 13>&-""" + (TEMP_PATH / "server_test.sh").write_text(server_test, encoding="utf-8") + (TEMP_PATH / "keeper_test.sh").write_text(keeper_test, encoding="utf-8") + + +def test_install_deb(image: DockerImage) -> TestResults: + tests = { + "Install server deb": r"""#!/bin/bash -ex +apt-get install /packages/clickhouse-{server,client,common}*deb +bash -ex /packages/server_test.sh""", + "Install keeper deb": r"""#!/bin/bash -ex +apt-get install /packages/clickhouse-keeper*deb +bash -ex /packages/keeper_test.sh""", + } + return test_install(image, tests) + + +def test_install_rpm(image: DockerImage) -> TestResults: + # FIXME: I couldn't find why Type=notify is broken in centos:8 + # systemd just ignores the watchdog completely + tests = { + "Install server rpm": r"""#!/bin/bash -ex +yum localinstall --disablerepo=* -y /packages/clickhouse-{server,client,common}*rpm +echo CLICKHOUSE_WATCHDOG_ENABLE=0 > /etc/default/clickhouse-server +bash -ex /packages/server_test.sh""", + "Install keeper rpm": r"""#!/bin/bash -ex +yum localinstall --disablerepo=* -y /packages/clickhouse-keeper*rpm +bash -ex /packages/keeper_test.sh""", + } + return test_install(image, tests) + + +def test_install_tgz(image: DockerImage) -> TestResults: + # FIXME: I couldn't find why Type=notify is broken in centos:8 + # systemd just ignores the watchdog completely + tests = { + f"Install server tgz in {image.name}": r"""#!/bin/bash -ex +[ -f /etc/debian_version ] && CONFIGURE=configure || CONFIGURE= +for pkg in /packages/clickhouse-{common,client,server}*tgz; do + package=${pkg%-*} + package=${package##*/} + tar xf "$pkg" + "/$package/install/doinst.sh" $CONFIGURE +done +[ -f /etc/yum.conf ] && echo CLICKHOUSE_WATCHDOG_ENABLE=0 > /etc/default/clickhouse-server +bash -ex /packages/server_test.sh""", + f"Install keeper tgz in {image.name}": r"""#!/bin/bash -ex +[ -f /etc/debian_version ] && CONFIGURE=configure || CONFIGURE= +for pkg in /packages/clickhouse-keeper*tgz; do + package=${pkg%-*} + package=${package##*/} + tar xf "$pkg" + "/$package/install/doinst.sh" $CONFIGURE +done +bash -ex /packages/keeper_test.sh""", + } + return test_install(image, tests) + + +def test_install(image: DockerImage, tests: Dict[str, str]) -> TestResults: + test_results = [] # type: TestResults + for name, command in tests.items(): + stopwatch = Stopwatch() + container_name = name.lower().replace(" ", "_").replace("/", "_") + log_file = TEMP_PATH / f"{container_name}.log" + run_command = ( + f"docker run --rm --privileged --detach --cap-add=SYS_PTRACE " + f"--volume={TEMP_PATH}:/packages {image}" + ) + logging.info("Running docker container: `%s`", run_command) + container_id = subprocess.check_output( + run_command, shell=True, encoding="utf-8" + ).strip() + (TEMP_PATH / "install.sh").write_text(command) + install_command = f"docker exec {container_id} bash -ex /packages/install.sh" + with TeePopen(install_command, log_file) as process: + retcode = process.wait() + if retcode == 0: + status = SUCCESS + else: + status = FAILURE + + subprocess.check_call(f"docker kill -s 9 {container_id}", shell=True) + test_results.append( + TestResult(name, status, stopwatch.duration_seconds, [log_file]) + ) + + return test_results + + +def parse_args() -> argparse.Namespace: + parser = argparse.ArgumentParser( + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + description="The script to check if the packages are able to install", + ) + + parser.add_argument( + "check_name", + help="check name, used to download the packages", + ) + parser.add_argument("--download", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-download", + dest="download", + action="store_false", + default=argparse.SUPPRESS, + help="if set, the packages won't be downloaded, useful for debug", + ) + parser.add_argument("--deb", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-deb", + dest="deb", + action="store_false", + default=argparse.SUPPRESS, + help="if set, the deb packages won't be checked", + ) + parser.add_argument("--rpm", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-rpm", + dest="rpm", + action="store_false", + default=argparse.SUPPRESS, + help="if set, the rpm packages won't be checked", + ) + parser.add_argument("--tgz", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-tgz", + dest="tgz", + action="store_false", + default=argparse.SUPPRESS, + help="if set, the tgz packages won't be checked", + ) + + return parser.parse_args() + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + args = parse_args() + + TEMP_PATH.mkdir(parents=True, exist_ok=True) + + pr_info = PRInfo() + + gh = Github(get_best_robot_token(), per_page=100) + + atexit.register(update_mergeable_check, gh, pr_info, args.check_name) + + rerun_helper = RerunHelper(gh, pr_info, args.check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + docker_images = { + name: get_image_with_version(REPORTS_PATH, name) + for name in (RPM_IMAGE, DEB_IMAGE) + } + prepare_test_scripts() + + if args.download: + + def filter_artifacts(path: str) -> bool: + return ( + path.endswith(".deb") + or path.endswith(".rpm") + or path.endswith(".tgz") + or path.endswith("/clickhouse") + ) + + download_builds_filter( + args.check_name, REPORTS_PATH, TEMP_PATH, filter_artifacts + ) + + test_results = [] # type: TestResults + if args.deb: + test_results.extend(test_install_deb(docker_images[DEB_IMAGE])) + if args.rpm: + test_results.extend(test_install_rpm(docker_images[RPM_IMAGE])) + if args.tgz: + test_results.extend(test_install_tgz(docker_images[DEB_IMAGE])) + test_results.extend(test_install_tgz(docker_images[RPM_IMAGE])) + + state = SUCCESS + description = "Packages installed successfully" + if FAILURE in (result.status for result in test_results): + state = FAILURE + description = "Failed to install packages: " + ", ".join( + result.name for result in test_results + ) + + s3_helper = S3Helper() + + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, args.check_name, test_results) + + report_url = upload_results( + s3_helper, + pr_info.number, + pr_info.sha, + test_results, + [], + args.check_name, + ) + print(f"::notice ::Report url: {report_url}") + if len(description) >= 140: + description = description[:136] + "..." + + post_commit_status(gh, pr_info.sha, args.check_name, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + test_results, + state, + stopwatch.duration_seconds, + stopwatch.start_time_str, + report_url, + args.check_name, + ) + + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + if state == FAILURE: + sys.exit(1) + + +if __name__ == "__main__": + main() From fc48867bedf2723bf8e2148e75ad850b1a3b3910 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 26 Jan 2023 17:16:05 +0100 Subject: [PATCH 62/78] Get rid of odd entity --- tests/ci/unit_tests_check.py | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 915a77f3d48..edc096908f4 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -108,13 +108,10 @@ def main(): stopwatch = Stopwatch() - temp_path = TEMP_PATH - reports_path = REPORTS_PATH - check_name = sys.argv[1] - if not os.path.exists(temp_path): - os.makedirs(temp_path) + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) pr_info = PRInfo() @@ -127,14 +124,14 @@ def main(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) - docker_image = get_image_with_version(reports_path, IMAGE_NAME) + docker_image = get_image_with_version(REPORTS_PATH, IMAGE_NAME) - download_unit_tests(check_name, reports_path, temp_path) + download_unit_tests(check_name, REPORTS_PATH, TEMP_PATH) - tests_binary_path = os.path.join(temp_path, "unit_tests_dbms") + tests_binary_path = os.path.join(TEMP_PATH, "unit_tests_dbms") os.chmod(tests_binary_path, 0o777) - test_output = os.path.join(temp_path, "test_output") + test_output = os.path.join(TEMP_PATH, "test_output") if not os.path.exists(test_output): os.makedirs(test_output) @@ -151,7 +148,7 @@ def main(): else: logging.info("Run failed") - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {TEMP_PATH}", shell=True) s3_helper = S3Helper() state, description, test_results, additional_logs = process_results(test_output) From a18a4a45023ab64f93a5e1daa4933bda7d94c68e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 27 Jan 2023 16:10:10 +0100 Subject: [PATCH 63/78] Use Path in download helper, do not redownload exist files --- tests/ci/build_download_helper.py | 27 +++++++++++++++++++-------- tests/ci/download_binary.py | 8 +++++--- 2 files changed, 24 insertions(+), 11 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index bd96ea04c4c..c6136015316 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -5,7 +5,8 @@ import logging import os import sys import time -from typing import Any, List, Optional +from pathlib import Path +from typing import Any, Callable, List, Optional import requests # type: ignore @@ -56,21 +57,29 @@ def read_build_urls(build_name: str, reports_path: str) -> List[str]: return [] -def download_build_with_progress(url, path): +def download_build_with_progress(url: str, path: Path) -> None: logging.info("Downloading from %s to temp path %s", url, path) for i in range(DOWNLOAD_RETRIES_COUNT): try: + response = get_with_retries(url, retries=1, stream=True) + total_length = int(response.headers.get("content-length", 0)) + if path.is_file() and total_length and path.stat().st_size == total_length: + logging.info( + "The file %s already exists and have a proper size %s", + path, + total_length, + ) + return + with open(path, "wb") as f: - response = get_with_retries(url, retries=1, stream=True) - total_length = response.headers.get("content-length") - if total_length is None or int(total_length) == 0: + if total_length == 0: logging.info( "No content-length, will download file without progress" ) f.write(response.content) else: dl = 0 - total_length = int(total_length) + logging.info("Content length is %ld bytes", total_length) for data in response.iter_content(chunk_size=4096): dl += len(data) @@ -99,12 +108,14 @@ def download_build_with_progress(url, path): logging.info("Downloading finished") -def download_builds(result_path, build_urls, filter_fn): +def download_builds( + result_path: str, build_urls: List[str], filter_fn: Callable[[str], bool] +) -> None: for url in build_urls: if filter_fn(url): fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) logging.info("Will download %s to %s", fname, result_path) - download_build_with_progress(url, os.path.join(result_path, fname)) + download_build_with_progress(url, Path(result_path) / fname) def download_builds_filter( diff --git a/tests/ci/download_binary.py b/tests/ci/download_binary.py index b95c86aa0bd..c57780daa36 100755 --- a/tests/ci/download_binary.py +++ b/tests/ci/download_binary.py @@ -6,6 +6,7 @@ This file is needed to avoid cicle import build_download_helper.py <=> env_helpe import argparse import logging import os +from pathlib import Path from build_download_helper import download_build_with_progress from ci_config import CI_CONFIG, BuildConfig @@ -57,14 +58,15 @@ def parse_args() -> argparse.Namespace: def main(): logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") args = parse_args() - os.makedirs(TEMP_PATH, exist_ok=True) + temp_path = Path(TEMP_PATH) + temp_path.mkdir(parents=True, exist_ok=True) for build in args.build_names: # check if it's in CI_CONFIG config = CI_CONFIG["build_config"][build] # type: BuildConfig if args.rename: - path = os.path.join(TEMP_PATH, f"clickhouse-{config['static_binary_name']}") + path = temp_path / f"clickhouse-{config['static_binary_name']}" else: - path = os.path.join(TEMP_PATH, "clickhouse") + path = temp_path / "clickhouse" url = S3_ARTIFACT_DOWNLOAD_TEMPLATE.format( pr_or_release=f"{args.version.major}.{args.version.minor}", From 360ea59ecfc070452b3cc867c3d3b3ec68343654 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 27 Jan 2023 23:36:06 +0100 Subject: [PATCH 64/78] Remove unused variables from clickhouse-server.postinstall --- packages/clickhouse-server.postinstall | 2 -- 1 file changed, 2 deletions(-) diff --git a/packages/clickhouse-server.postinstall b/packages/clickhouse-server.postinstall index ff376b89bd4..2b9830faf3b 100644 --- a/packages/clickhouse-server.postinstall +++ b/packages/clickhouse-server.postinstall @@ -11,8 +11,6 @@ CLICKHOUSE_DATADIR=${CLICKHOUSE_DATADIR:=/var/lib/clickhouse} CLICKHOUSE_LOGDIR=${CLICKHOUSE_LOGDIR:=/var/log/clickhouse-server} CLICKHOUSE_BINDIR=${CLICKHOUSE_BINDIR:=/usr/bin} CLICKHOUSE_GENERIC_PROGRAM=${CLICKHOUSE_GENERIC_PROGRAM:=clickhouse} -EXTRACT_FROM_CONFIG=${CLICKHOUSE_GENERIC_PROGRAM}-extract-from-config -CLICKHOUSE_CONFIG=$CLICKHOUSE_CONFDIR/config.xml CLICKHOUSE_PIDDIR=/var/run/$PROGRAM [ -f /usr/share/debconf/confmodule ] && . /usr/share/debconf/confmodule From 496701bd5970c043a511d12726a44a4c71471b0b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sat, 28 Jan 2023 00:00:01 +0100 Subject: [PATCH 65/78] Add necessary postinst steps for clickhouse-keeper --- packages/clickhouse-keeper.postinstall | 46 ++++++++++++++++++++++++++ packages/clickhouse-keeper.yaml | 3 ++ 2 files changed, 49 insertions(+) create mode 100644 packages/clickhouse-keeper.postinstall diff --git a/packages/clickhouse-keeper.postinstall b/packages/clickhouse-keeper.postinstall new file mode 100644 index 00000000000..3d6cd484146 --- /dev/null +++ b/packages/clickhouse-keeper.postinstall @@ -0,0 +1,46 @@ +#!/bin/sh +set -e +# set -x + +PROGRAM=clickhouse-keeper +KEEPER_USER=${KEEPER_USER:=clickhouse} +KEEPER_GROUP=${KEEPER_GROUP:=clickhouse} +# Please note that we don't support paths with whitespaces. This is rather ignorant. +KEEPER_CONFDIR=${KEEPER_CONFDIR:=/etc/$PROGRAM} +KEEPER_DATADIR=${KEEPER_DATADIR:=/var/lib/clickhouse} +KEEPER_LOGDIR=${KEEPER_LOGDIR:=/var/log/$PROGRAM} + +[ -f /usr/share/debconf/confmodule ] && . /usr/share/debconf/confmodule +[ -f /etc/default/clickhouse-keeper ] && . /etc/default/clickhouse-keeper + +if [ ! -f "/etc/debian_version" ]; then + not_deb_os=1 +fi + +if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then + if ! getent group "${KEEPER_GROUP}" > /dev/null 2>&1 ; then + groupadd --system "${KEEPER_GROUP}" + fi + GID=$(getent group "${KEEPER_GROUP}" | cut -d: -f 3) + if ! id "${KEEPER_USER}" > /dev/null 2>&1 ; then + adduser --system --home /dev/null --no-create-home \ + --gid "${GID}" --shell /bin/false \ + "${KEEPER_USER}" + fi + + chown -R "${KEEPER_USER}:${KEEPER_GROUP}" "${KEEPER_CONFDIR}" + chmod 0755 "${KEEPER_CONFDIR}" + + if ! [ -d "${KEEPER_DATADIR}" ]; then + mkdir -p "${KEEPER_DATADIR}" + chown -R "${KEEPER_USER}:${KEEPER_GROUP}" "${KEEPER_DATADIR}" + chmod 0700 "${KEEPER_DATADIR}" + fi + + if ! [ -d "${KEEPER_LOGDIR}" ]; then + mkdir -p "${KEEPER_LOGDIR}" + chown -R "${KEEPER_USER}:${KEEPER_GROUP}" "${KEEPER_LOGDIR}" + chmod 0770 "${KEEPER_LOGDIR}" + fi +fi +# vim: ts=4: sw=4: sts=4: expandtab diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml index 0706dc3fe7e..e9c2e929755 100644 --- a/packages/clickhouse-keeper.yaml +++ b/packages/clickhouse-keeper.yaml @@ -41,3 +41,6 @@ contents: dst: /usr/share/doc/clickhouse-keeper/LICENSE - src: ../README.md dst: /usr/share/doc/clickhouse-keeper/README.md + +scripts: + postinstall: ./clickhouse-keeper.postinstall From a42d20e19d9dd53f9c5ea853cdb53d424ee46f19 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Jan 2023 15:26:57 +0100 Subject: [PATCH 66/78] Create "coordination" directory in keeper if does not exist --- programs/keeper/Keeper.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3a0d3d3a6ca..ed3297ed7cb 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -362,6 +362,7 @@ try else path = std::filesystem::path{KEEPER_DEFAULT_PATH}; + std::filesystem::create_directories(path); /// Check that the process user id matches the owner of the data. const auto effective_user_id = geteuid(); From 7a7b16dd139fd77c0b729323b95bdecb77116347 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 31 Jan 2023 20:31:42 +0100 Subject: [PATCH 67/78] Do not use debconf/confmodule in tgz packages --- packages/build | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/build b/packages/build index c8fb77e9371..6ec991aad07 100755 --- a/packages/build +++ b/packages/build @@ -102,7 +102,8 @@ done EOF chmod +x "$PKG_PATH/install/doinst.sh" if [ -f "$PKG_PATH/DEBIAN/postinst" ]; then - tail +2 "$PKG_PATH/DEBIAN/postinst" >> "$PKG_PATH/install/doinst.sh" + # we don't need debconf source in doinst in any case + tail +2 "$PKG_PATH/DEBIAN/postinst" | grep -v debconf/confmodule >> "$PKG_PATH/install/doinst.sh" fi rm -rf "$PKG_PATH/DEBIAN" if [ -f "/usr/bin/pigz" ]; then From e41a78b67a7f2d0cf1667f71c5bef9616217e39d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 1 Feb 2023 13:11:39 +0100 Subject: [PATCH 68/78] Improve local tests --- tests/ci/install_check.py | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index e45a4fffe96..1c4f7672a18 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -20,7 +20,7 @@ from clickhouse_helper import ( ) from commit_status_helper import post_commit_status, update_mergeable_check from docker_pull_helper import get_image_with_version, DockerImage -from env_helper import TEMP_PATH as TEMP, REPORTS_PATH +from env_helper import CI, TEMP_PATH as TEMP, REPORTS_PATH from get_robot_token import get_best_robot_token from pr_info import PRInfo from report import TestResults, TestResult @@ -202,14 +202,16 @@ def main(): pr_info = PRInfo() - gh = Github(get_best_robot_token(), per_page=100) + if CI: + gh = Github(get_best_robot_token(), per_page=100) + atexit.register(update_mergeable_check, gh, pr_info, args.check_name) - atexit.register(update_mergeable_check, gh, pr_info, args.check_name) - - rerun_helper = RerunHelper(gh, pr_info, args.check_name) - if rerun_helper.is_already_finished_by_status(): - logging.info("Check is already finished according to github status, exiting") - sys.exit(0) + rerun_helper = RerunHelper(gh, pr_info, args.check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info( + "Check is already finished according to github status, exiting" + ) + sys.exit(0) docker_images = { name: get_image_with_version(REPORTS_PATH, name) @@ -250,9 +252,6 @@ def main(): s3_helper = S3Helper() - ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) - report_url = upload_results( s3_helper, pr_info.number, @@ -262,6 +261,12 @@ def main(): args.check_name, ) print(f"::notice ::Report url: {report_url}") + if not CI: + return + + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, args.check_name, test_results) + if len(description) >= 140: description = description[:136] + "..." From 981101a40a10f5811153916052a49b5272882133 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 1 Feb 2023 23:33:05 +0100 Subject: [PATCH 69/78] Add arm64 installation tests --- .github/workflows/pull_request.yml | 35 +++++++++++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index be8c99adc43..527a849621d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -987,7 +987,7 @@ jobs: ############################################################################################ #################################### INSTALL PACKAGES ###################################### ############################################################################################ - TestInstallPackagesRelease: + InstallPackagesTestRelease: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: @@ -1020,6 +1020,39 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + InstallPackagesTestAarch64: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/test_install + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Install packages (arm64) + REPO_COPY=${{runner.temp}}/test_install/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Test packages installation + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 install_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## From dd64c8bacca0eb74239466a04760e21561207c9a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Feb 2023 11:51:09 +0100 Subject: [PATCH 70/78] Add tests for installation from binary --- tests/ci/install_check.py | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 1c4f7672a18..1444759cea0 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -39,10 +39,10 @@ FAILURE = "failure" def prepare_test_scripts(): - server_test = """#!/bin/bash + server_test = r"""#!/bin/bash systemctl start clickhouse-server clickhouse-client -q 'SELECT version()'""" - keeper_test = """#!/bin/bash + keeper_test = r"""#!/bin/bash systemctl start clickhouse-keeper for i in {1..20}; do echo wait for clickhouse-keeper to being up @@ -55,9 +55,30 @@ for i in {1..5}; do cat <&13 | grep zk_version && break || sleep 1 exec 13>&- done +exec 13>&-""" + binary_test = r"""#!/bin/bash +chmod +x /packages/clickhouse +/packages/clickhouse install +clickhouse-server start --daemon +for i in {1..5}; do + clickhouse-client -q 'SELECT version()' && break || sleep 1 +done +clickhouse-keeper start --daemon +for i in {1..20}; do + echo wait for clickhouse-keeper to being up + > /dev/tcp/127.0.0.1/9181 2>/dev/null && break || sleep 1 +done +for i in {1..5}; do + echo wait for clickhouse-keeper to answer on mntr request + exec 13<>/dev/tcp/127.0.0.1/9181 + echo mntr >&13 + cat <&13 | grep zk_version && break || sleep 1 + exec 13>&- +done exec 13>&-""" (TEMP_PATH / "server_test.sh").write_text(server_test, encoding="utf-8") (TEMP_PATH / "keeper_test.sh").write_text(keeper_test, encoding="utf-8") + (TEMP_PATH / "binary_test.sh").write_text(binary_test, encoding="utf-8") def test_install_deb(image: DockerImage) -> TestResults: @@ -68,6 +89,7 @@ bash -ex /packages/server_test.sh""", "Install keeper deb": r"""#!/bin/bash -ex apt-get install /packages/clickhouse-keeper*deb bash -ex /packages/keeper_test.sh""", + "Install clickhouse binary in deb": r"bash -ex /packages/binary_test.sh", } return test_install(image, tests) @@ -83,6 +105,7 @@ bash -ex /packages/server_test.sh""", "Install keeper rpm": r"""#!/bin/bash -ex yum localinstall --disablerepo=* -y /packages/clickhouse-keeper*rpm bash -ex /packages/keeper_test.sh""", + "Install clickhouse binary in rpm": r"bash -ex /packages/binary_test.sh", } return test_install(image, tests) From d9d51934a9d3f2810258fc51cc822304e9256141 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 3 Feb 2023 11:27:32 +0000 Subject: [PATCH 71/78] Docs: Mention time zone randomization --- docs/en/development/tests.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 0b26de57326..28aee32c717 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -16,6 +16,11 @@ Tests are located in `queries` directory. There are two subdirectories: `statele Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. SQL tests are generally preferable to `.sh` tests. You should use `.sh` tests only when you have to test some feature that cannot be exercised from pure SQL, such as piping some input data into `clickhouse-client` or testing `clickhouse-local`. +:::note +A common mistake when testing data types `DateTime` and `DateTime64` is assuming that the server uses a specific time zone (e.g. "UTC"). This is not the case, time zones in CI test runs +are deliberately randomized. The easiest workaround is to specify the time zone for test values explicitly, e.g. `toDateTime64(val, 3, 'Europe/Amsterdam')`. +::: + ### Running a Test Locally {#functional-test-locally} Start the ClickHouse server locally, listening on the default port (9000). To From a9ab22e45dad60a5da2c864c6dde5ca5be95cd88 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 3 Feb 2023 13:00:13 +0000 Subject: [PATCH 72/78] Update version_date.tsv and changelogs after v23.1.3.5-stable --- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.1.3.5-stable.md | 17 +++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 4 files changed, 20 insertions(+), 2 deletions(-) create mode 100644 docs/changelogs/v23.1.3.5-stable.md diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 522fd354393..8a73d72b3a5 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.1.2.9" +ARG VERSION="23.1.3.5" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 434b4c3bff0..ba2d7430e06 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -21,7 +21,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.1.2.9" +ARG VERSION="23.1.3.5" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.1.3.5-stable.md b/docs/changelogs/v23.1.3.5-stable.md new file mode 100644 index 00000000000..d4f39894bec --- /dev/null +++ b/docs/changelogs/v23.1.3.5-stable.md @@ -0,0 +1,17 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.1.3.5-stable (548b494bcce) FIXME as compared to v23.1.2.9-stable (8dfb1700858) + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#45896](https://github.com/ClickHouse/ClickHouse/issues/45896): Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4535eeaf243..e09a39ff463 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 v22.12.3.5-stable 2023-01-10 From d5117f2aa6e1c313568618f199a86ac7ebd7f94b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 Feb 2023 14:30:52 +0100 Subject: [PATCH 73/78] Define S3 client with bucket and endpoint resolution (#45783) * Update aws * Define S3 client with bucket and endpoint resolution * Add defines for ErrorCodes * Use S3Client everywhere * Remove unused errorcode * Add DROP S3 CLIENT CACHE query * Add a comment * Fix style * Update aws * Update reference files * Add missing include * Fix unit test * Remove unneeded declarations * Correctly use RetryStrategy * Rename S3Client to Client * Fix retry count * fix clang-tidy warnings --- contrib/aws | 2 +- src/Access/Common/AccessType.h | 1 + src/Backups/BackupIO_S3.cpp | 17 +- src/Backups/BackupIO_S3.h | 5 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 9 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 18 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 11 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/Disks/ObjectStorages/S3/diskSettings.h | 6 +- src/IO/ReadBufferFromS3.cpp | 8 +- src/IO/ReadBufferFromS3.h | 10 +- src/IO/S3/Client.cpp | 400 ++++++++++++++++++ src/IO/S3/Client.h | 309 ++++++++++++++ src/IO/S3/Requests.h | 87 ++++ src/IO/S3/URI.cpp | 119 ++++++ src/IO/S3/URI.h | 40 ++ src/IO/S3/copyS3File.cpp | 48 +-- src/IO/S3/copyS3File.h | 5 +- src/IO/S3/getObjectInfo.cpp | 109 +---- src/IO/S3/getObjectInfo.h | 10 +- src/IO/S3/tests/gtest_aws_s3_client.cpp | 4 +- src/IO/S3Common.cpp | 114 +---- src/IO/S3Common.h | 34 +- src/IO/WriteBufferFromS3.cpp | 24 +- src/IO/WriteBufferFromS3.h | 17 +- src/Interpreters/InterpreterSystemQuery.cpp | 14 + src/Parsers/ASTSystemQuery.h | 3 + src/Storages/StorageDeltaLake.cpp | 5 +- src/Storages/StorageDeltaLake.h | 5 - src/Storages/StorageHudi.cpp | 5 +- src/Storages/StorageHudi.h | 5 - src/Storages/StorageS3.cpp | 28 +- src/Storages/StorageS3.h | 12 +- src/Storages/StorageS3Cluster.cpp | 2 - .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 +- 36 files changed, 1115 insertions(+), 380 deletions(-) create mode 100644 src/IO/S3/Client.cpp create mode 100644 src/IO/S3/Client.h create mode 100644 src/IO/S3/Requests.h create mode 100644 src/IO/S3/URI.cpp create mode 100644 src/IO/S3/URI.h diff --git a/contrib/aws b/contrib/aws index 4a12641211d..06a6610e6fb 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 4a12641211d4dbc8e2fdb2dd0f1eea0927db9252 +Subproject commit 06a6610e6fb3385e22ad85014a67aa307825ffb1 diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 88ab3704358..2698ce0d9f7 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -146,6 +146,7 @@ enum class AccessType M(SYSTEM_DROP_COMPILED_EXPRESSION_CACHE, "SYSTEM DROP COMPILED EXPRESSION, DROP COMPILED EXPRESSION CACHE, DROP COMPILED EXPRESSIONS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_FILESYSTEM_CACHE, "SYSTEM DROP FILESYSTEM CACHE, DROP FILESYSTEM CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \ M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_USERS, "RELOAD USERS", GLOBAL, SYSTEM_RELOAD) \ diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 0cca1ac6aac..a303a0bc2d5 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -9,13 +9,12 @@ #include #include #include +#include + #include #include -#include -#include -#include -#include + #include @@ -31,7 +30,7 @@ namespace ErrorCodes namespace { - std::shared_ptr + std::shared_ptr makeS3Client(const S3::URI & s3_uri, const String & access_key_id, const String & secret_access_key, const ContextPtr & context) { auto settings = context->getStorageS3Settings().getSettings(s3_uri.uri.toString()); @@ -71,9 +70,9 @@ namespace context->getConfigRef().getBool("s3.use_insecure_imds_request", false))); } - Aws::Vector listObjects(Aws::S3::S3Client & client, const S3::URI & s3_uri, const String & file_name) + Aws::Vector listObjects(S3::Client & client, const S3::URI & s3_uri, const String & file_name) { - Aws::S3::Model::ListObjectsRequest request; + S3::ListObjectsRequest request; request.SetBucket(s3_uri.bucket); request.SetPrefix(fs::path{s3_uri.key} / file_name); request.SetMaxKeys(1); @@ -228,7 +227,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) void BackupWriterS3::removeFile(const String & file_name) { - Aws::S3::Model::DeleteObjectRequest request; + S3::DeleteObjectRequest request; request.SetBucket(s3_uri.bucket); request.SetKey(fs::path(s3_uri.key) / file_name); auto outcome = client->DeleteObject(request); @@ -285,7 +284,7 @@ void BackupWriterS3::removeFilesBatch(const Strings & file_names) Aws::S3::Model::Delete delkeys; delkeys.SetObjects(current_chunk); - Aws::S3::Model::DeleteObjectsRequest request; + S3::DeleteObjectsRequest request; request.SetBucket(s3_uri.bucket); request.SetDelete(delkeys); diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 41925eb4dcd..9c3132c5689 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB @@ -27,7 +26,7 @@ public: private: S3::URI s3_uri; - std::shared_ptr client; + std::shared_ptr client; ReadSettings read_settings; S3Settings::RequestSettings request_settings; }; @@ -73,7 +72,7 @@ private: void removeFilesBatch(const Strings & file_names); S3::URI s3_uri; - std::shared_ptr client; + std::shared_ptr client; ReadSettings read_settings; S3Settings::RequestSettings request_settings; Poco::Logger * log; diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index c4cbbb7f28a..7b47324a890 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -17,10 +17,7 @@ #include #include -#include #include -#include -#include #include @@ -31,7 +28,7 @@ namespace DB struct KeeperSnapshotManagerS3::S3Configuration { - S3Configuration(S3::URI uri_, S3::AuthSettings auth_settings_, std::shared_ptr client_) + S3Configuration(S3::URI uri_, S3::AuthSettings auth_settings_, std::shared_ptr client_) : uri(std::move(uri_)) , auth_settings(std::move(auth_settings_)) , client(std::move(client_)) @@ -39,7 +36,7 @@ struct KeeperSnapshotManagerS3::S3Configuration S3::URI uri; S3::AuthSettings auth_settings; - std::shared_ptr client; + std::shared_ptr client; }; KeeperSnapshotManagerS3::KeeperSnapshotManagerS3() @@ -202,7 +199,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa LOG_INFO(log, "Removing lock file"); try { - Aws::S3::Model::DeleteObjectRequest delete_request; + S3::DeleteObjectRequest delete_request; delete_request.SetBucket(s3_client->uri.bucket); delete_request.SetKey(lock_file); auto delete_outcome = s3_client->client->DeleteObject(delete_request); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index fca8f4111b3..cbba5ed64f9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -21,10 +21,6 @@ #include #include -#include -#include -#include - #include #include #include @@ -213,7 +209,7 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); - Aws::S3::Model::ListObjectsV2Request request; + S3::ListObjectsV2Request request; request.SetBucket(bucket); request.SetPrefix(path); if (max_keys) @@ -257,7 +253,7 @@ void S3ObjectStorage::getDirectoryContents(const std::string & path, auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); - Aws::S3::Model::ListObjectsV2Request request; + S3::ListObjectsV2Request request; request.SetBucket(bucket); /// NOTE: if you do "ls /foo" instead of "ls /foo/" over S3 with this API /// it will return only "/foo" itself without any underlying nodes. @@ -304,7 +300,7 @@ void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exis ProfileEvents::increment(ProfileEvents::S3DeleteObjects); ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects); - Aws::S3::Model::DeleteObjectRequest request; + S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(object.absolute_path); auto outcome = client_ptr->DeleteObject(request); @@ -352,7 +348,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e ProfileEvents::increment(ProfileEvents::S3DeleteObjects); ProfileEvents::increment(ProfileEvents::DiskS3DeleteObjects); - Aws::S3::Model::DeleteObjectsRequest request; + S3::DeleteObjectsRequest request; request.SetBucket(bucket); request.SetDelete(delkeys); auto outcome = client_ptr->DeleteObjects(request); @@ -435,7 +431,7 @@ void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings.set(std::move(s3_settings_)); } -void S3ObjectStorage::setNewClient(std::unique_ptr && client_) +void S3ObjectStorage::setNewClient(std::unique_ptr && client_) { client.set(std::move(client_)); } @@ -447,7 +443,7 @@ void S3ObjectStorage::shutdown() /// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome. /// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors. /// This should significantly speed up shutdown process if S3 is unhealthy. - const_cast(*client_ptr).DisableRequestProcessing(); + const_cast(*client_ptr).DisableRequestProcessing(); } void S3ObjectStorage::startup() @@ -455,7 +451,7 @@ void S3ObjectStorage::startup() auto client_ptr = client.get(); /// Need to be enabled if it was disabled during shutdown() call. - const_cast(*client_ptr).EnableRequestProcessing(); + const_cast(*client_ptr).EnableRequestProcessing(); } void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index ba3eeb421c4..bcdc97983be 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -46,7 +45,7 @@ private: S3ObjectStorage( const char * logger_name, - std::unique_ptr && client_, + std::unique_ptr && client_, std::unique_ptr && s3_settings_, String version_id_, const S3Capabilities & s3_capabilities_, @@ -68,7 +67,7 @@ private: public: template - S3ObjectStorage(std::unique_ptr && client_, Args && ...args) + explicit S3ObjectStorage(std::unique_ptr && client_, Args && ...args) : S3ObjectStorage("S3ObjectStorage", std::move(client_), std::forward(args)...) { } @@ -163,14 +162,14 @@ public: private: void setNewSettings(std::unique_ptr && s3_settings_); - void setNewClient(std::unique_ptr && client_); + void setNewClient(std::unique_ptr && client_); void removeObjectImpl(const StoredObject & object, bool if_exists); void removeObjectsImpl(const StoredObjects & objects, bool if_exists); std::string bucket; - MultiVersion client; + MultiVersion client; MultiVersion s3_settings; S3Capabilities s3_capabilities; @@ -191,7 +190,7 @@ public: std::string getName() const override { return "S3PlainObjectStorage"; } template - S3PlainObjectStorage(Args && ...args) + explicit S3PlainObjectStorage(Args && ...args) : S3ObjectStorage("S3PlainObjectStorage", std::forward(args)...) { data_source_description.type = DataSourceType::S3_Plain; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4cabe9461a3..e0e4735f519 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -107,7 +107,7 @@ std::shared_ptr getProxyConfiguration(const String & pre } -std::unique_ptr getClient( +std::unique_ptr getClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 04eb7aced8e..83bf7b179ef 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -7,11 +7,13 @@ #include #include +#include + namespace Aws { namespace S3 { -class S3Client; +class Client; } } @@ -22,7 +24,7 @@ struct S3ObjectStorageSettings; std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); +std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 6cc98464af2..8e48c3044a3 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -7,9 +7,7 @@ #include #include -#include -#include -#include +#include #include #include @@ -44,7 +42,7 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, const String & version_id_, @@ -281,7 +279,7 @@ SeekableReadBuffer::Range ReadBufferFromS3::getRemainingReadRange() const std::unique_ptr ReadBufferFromS3::initialize() { - Aws::S3::Model::GetObjectRequest req; + S3::GetObjectRequest req; req.SetBucket(bucket); req.SetKey(key); if (!version_id.empty()) diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 17b13bf7d62..84e8d36865c 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -19,7 +19,7 @@ namespace Aws::S3 { -class S3Client; +class Client; } namespace DB @@ -30,7 +30,7 @@ namespace DB class ReadBufferFromS3 : public ReadBufferFromFileBase { private: - std::shared_ptr client_ptr; + std::shared_ptr client_ptr; String bucket; String key; String version_id; @@ -49,7 +49,7 @@ private: public: ReadBufferFromS3( - std::shared_ptr client_ptr_, + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, const String & version_id_, @@ -95,7 +95,7 @@ class ReadBufferS3Factory : public ParallelReadBuffer::ReadBufferFactory, public { public: explicit ReadBufferS3Factory( - std::shared_ptr client_ptr_, + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, const String & version_id_, @@ -126,7 +126,7 @@ public: String getFileName() const override { return bucket + "/" + key; } private: - std::shared_ptr client_ptr; + std::shared_ptr client_ptr; const String bucket; const String key; const String version_id; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp new file mode 100644 index 00000000000..2e07dfb19ff --- /dev/null +++ b/src/IO/S3/Client.cpp @@ -0,0 +1,400 @@ +#include + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace S3 +{ + +Client::RetryStrategy::RetryStrategy(std::shared_ptr wrapped_strategy_) + : wrapped_strategy(std::move(wrapped_strategy_)) +{ + if (!wrapped_strategy) + wrapped_strategy = Aws::Client::InitRetryStrategy(); +} + +/// NOLINTNEXTLINE(google-runtime-int) +bool Client::RetryStrategy::ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const +{ + if (error.GetResponseCode() == Aws::Http::HttpResponseCode::MOVED_PERMANENTLY) + return false; + + return wrapped_strategy->ShouldRetry(error, attemptedRetries); +} + +/// NOLINTNEXTLINE(google-runtime-int) +long Client::RetryStrategy::CalculateDelayBeforeNextRetry(const Aws::Client::AWSError& error, long attemptedRetries) const +{ + return wrapped_strategy->CalculateDelayBeforeNextRetry(error, attemptedRetries); +} + +/// NOLINTNEXTLINE(google-runtime-int) +long Client::RetryStrategy::GetMaxAttempts() const +{ + return wrapped_strategy->GetMaxAttempts(); +} + +void Client::RetryStrategy::GetSendToken() +{ + return wrapped_strategy->GetSendToken(); +} + +bool Client::RetryStrategy::HasSendToken() +{ + return wrapped_strategy->HasSendToken(); +} + +void Client::RetryStrategy::RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome) +{ + return wrapped_strategy->RequestBookkeeping(httpResponseOutcome); +} + +void Client::RetryStrategy::RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome, const Aws::Client::AWSError& lastError) +{ + return wrapped_strategy->RequestBookkeeping(httpResponseOutcome, lastError); +} + +bool Client::checkIfWrongRegionDefined(const std::string & bucket, const Aws::S3::S3Error & error, std::string & region) const +{ + if (detect_region) + return false; + + if (error.GetResponseCode() == Aws::Http::HttpResponseCode::BAD_REQUEST && error.GetExceptionName() == "AuthorizationHeaderMalformed") + { + region = GetErrorMarshaller()->ExtractRegion(error); + + if (region.empty()) + region = getRegionForBucket(bucket, /*force_detect*/ true); + + assert(!explicit_region.empty()); + if (region == explicit_region) + return false; + + insertRegionOverride(bucket, region); + return true; + } + + return false; +} + +void Client::insertRegionOverride(const std::string & bucket, const std::string & region) const +{ + std::lock_guard lock(cache->region_cache_mutex); + auto [it, inserted] = cache->region_for_bucket_cache.emplace(bucket, region); + if (inserted) + LOG_INFO(log, "Detected different region ('{}') for bucket {} than the one defined ('{}')", region, bucket, explicit_region); +} + +Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) const +{ + const auto & bucket = request.GetBucket(); + + if (auto region = getRegionForBucket(bucket); !region.empty()) + { + if (!detect_region) + LOG_INFO(log, "Using region override {} for bucket {}", region, bucket); + request.overrideRegion(std::move(region)); + } + + if (auto uri = getURIForBucket(bucket); uri.has_value()) + request.overrideURI(std::move(*uri)); + + auto result = Aws::S3::S3Client::HeadObject(request); + if (result.IsSuccess()) + return result; + + const auto & error = result.GetError(); + + std::string new_region; + if (checkIfWrongRegionDefined(bucket, error, new_region)) + { + request.overrideRegion(new_region); + return HeadObject(request); + } + + if (error.GetResponseCode() != Aws::Http::HttpResponseCode::MOVED_PERMANENTLY) + return result; + + // maybe we detect a correct region + if (!detect_region) + { + if (auto region = GetErrorMarshaller()->ExtractRegion(error); !region.empty() && region != explicit_region) + { + request.overrideRegion(region); + insertRegionOverride(bucket, region); + } + } + + auto bucket_uri = getURIForBucket(bucket); + if (!bucket_uri) + { + if (auto maybe_error = updateURIForBucketForHead(bucket); maybe_error.has_value()) + return *maybe_error; + + if (auto region = getRegionForBucket(bucket); !region.empty()) + { + if (!detect_region) + LOG_INFO(log, "Using region override {} for bucket {}", region, bucket); + request.overrideRegion(std::move(region)); + } + + bucket_uri = getURIForBucket(bucket); + if (!bucket_uri) + { + LOG_ERROR(log, "Missing resolved URI for bucket {}, maybe the cache was cleaned", bucket); + return result; + } + } + + const auto & current_uri_override = request.getURIOverride(); + /// we already tried with this URI + if (current_uri_override && current_uri_override->uri == bucket_uri->uri) + { + LOG_INFO(log, "Getting redirected to the same invalid location {}", bucket_uri->uri.toString()); + return result; + } + + request.overrideURI(std::move(*bucket_uri)); + + return Aws::S3::S3Client::HeadObject(request); +} + +Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const +{ + return doRequest(request, [this](const Model::ListObjectsV2Request & req) { return Aws::S3::S3Client::ListObjectsV2(req); }); +} + +Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const +{ + return doRequest(request, [this](const Model::ListObjectsRequest & req) { return Aws::S3::S3Client::ListObjects(req); }); +} + +Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const +{ + return doRequest(request, [this](const Model::GetObjectRequest & req) { return Aws::S3::S3Client::GetObject(req); }); +} + +Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const +{ + return doRequest( + request, [this](const Model::AbortMultipartUploadRequest & req) { return Aws::S3::S3Client::AbortMultipartUpload(req); }); +} + +Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const +{ + return doRequest( + request, [this](const Model::CreateMultipartUploadRequest & req) { return Aws::S3::S3Client::CreateMultipartUpload(req); }); +} + +Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const +{ + return doRequest( + request, [this](const Model::CompleteMultipartUploadRequest & req) { return Aws::S3::S3Client::CompleteMultipartUpload(req); }); +} + +Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const +{ + return doRequest(request, [this](const Model::CopyObjectRequest & req) { return Aws::S3::S3Client::CopyObject(req); }); +} + +Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const +{ + return doRequest(request, [this](const Model::PutObjectRequest & req) { return Aws::S3::S3Client::PutObject(req); }); +} + +Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const +{ + return doRequest(request, [this](const Model::UploadPartRequest & req) { return Aws::S3::S3Client::UploadPart(req); }); +} + +Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const +{ + return doRequest(request, [this](const Model::UploadPartCopyRequest & req) { return Aws::S3::S3Client::UploadPartCopy(req); }); +} + +Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const +{ + return doRequest(request, [this](const Model::DeleteObjectRequest & req) { return Aws::S3::S3Client::DeleteObject(req); }); +} + +Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const +{ + return doRequest(request, [this](const Model::DeleteObjectsRequest & req) { return Aws::S3::S3Client::DeleteObjects(req); }); +} + +std::string Client::getRegionForBucket(const std::string & bucket, bool force_detect) const +{ + std::lock_guard lock(cache->region_cache_mutex); + if (auto it = cache->region_for_bucket_cache.find(bucket); it != cache->region_for_bucket_cache.end()) + return it->second; + + if (!force_detect && !detect_region) + return ""; + + + LOG_INFO(log, "Resolving region for bucket {}", bucket); + Aws::S3::Model::HeadBucketRequest req; + req.SetBucket(bucket); + + std::string region; + auto outcome = HeadBucket(req); + if (outcome.IsSuccess()) + { + const auto & result = outcome.GetResult(); + region = result.GetRegion(); + } + else + { + static const std::string region_header = "x-amz-bucket-region"; + const auto & headers = outcome.GetError().GetResponseHeaders(); + if (auto it = headers.find(region_header); it != headers.end()) + region = it->second; + } + + if (region.empty()) + { + LOG_INFO(log, "Failed resolving region for bucket {}", bucket); + return ""; + } + + LOG_INFO(log, "Found region {} for bucket {}", region, bucket); + + auto [it, _] = cache->region_for_bucket_cache.emplace(bucket, std::move(region)); + + return it->second; +} + +std::optional Client::getURIFromError(const Aws::S3::S3Error & error) const +{ + auto endpoint = GetErrorMarshaller()->ExtractEndpoint(error); + if (endpoint.empty()) + return std::nullopt; + + auto & s3_client = const_cast(*this); + const auto * endpoint_provider = dynamic_cast(s3_client.accessEndpointProvider().get()); + auto resolved_endpoint = endpoint_provider->ResolveEndpoint({}); + + if (!resolved_endpoint.IsSuccess()) + return std::nullopt; + + auto uri = resolved_endpoint.GetResult().GetURI(); + uri.SetAuthority(endpoint); + + return S3::URI(uri.GetURIString()); +} + +// Do a list request because head requests don't have body in response +std::optional Client::updateURIForBucketForHead(const std::string & bucket) const +{ + ListObjectsV2Request req; + req.SetBucket(bucket); + req.SetMaxKeys(1); + auto result = ListObjectsV2(req); + if (result.IsSuccess()) + return std::nullopt; + return result.GetError(); +} + +std::optional Client::getURIForBucket(const std::string & bucket) const +{ + std::lock_guard lock(cache->uri_cache_mutex); + if (auto it = cache->uri_for_bucket_cache.find(bucket); it != cache->uri_for_bucket_cache.end()) + return it->second; + + return std::nullopt; +} + +void Client::updateURIForBucket(const std::string & bucket, S3::URI new_uri) const +{ + std::lock_guard lock(cache->uri_cache_mutex); + if (auto it = cache->uri_for_bucket_cache.find(bucket); it != cache->uri_for_bucket_cache.end()) + { + if (it->second.uri == new_uri.uri) + return; + + LOG_INFO(log, "Updating URI for bucket {} to {}", bucket, new_uri.uri.toString()); + it->second = std::move(new_uri); + + return; + } + + LOG_INFO(log, "Updating URI for bucket {} to {}", bucket, new_uri.uri.toString()); + cache->uri_for_bucket_cache.emplace(bucket, std::move(new_uri)); +} + + +void ClientCache::clearCache() +{ + { + std::lock_guard lock(region_cache_mutex); + region_for_bucket_cache.clear(); + } + { + std::lock_guard lock(uri_cache_mutex); + uri_for_bucket_cache.clear(); + } +} + +void ClientCacheRegistry::registerClient(const std::shared_ptr & client_cache) +{ + std::lock_guard lock(clients_mutex); + auto [it, inserted] = client_caches.emplace(client_cache.get(), client_cache); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Same S3 client registered twice"); +} + +void ClientCacheRegistry::unregisterClient(ClientCache * client) +{ + std::lock_guard lock(clients_mutex); + auto erased = client_caches.erase(client); + if (erased == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't unregister S3 client, either it was already unregistered or not registered at all"); +} + +void ClientCacheRegistry::clearCacheForAll() +{ + std::lock_guard lock(clients_mutex); + + for (auto it = client_caches.begin(); it != client_caches.end();) + { + if (auto locked_client = it->second.lock(); locked_client) + { + locked_client->clearCache(); + ++it; + } + else + { + LOG_INFO(&Poco::Logger::get("ClientCacheRegistry"), "Deleting leftover S3 client cache"); + it = client_caches.erase(it); + } + } + +} + +} + +} + +#endif diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h new file mode 100644 index 00000000000..13f26c214f2 --- /dev/null +++ b/src/IO/S3/Client.h @@ -0,0 +1,309 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int TOO_MANY_REDIRECTS; +} + +namespace S3 +{ + +namespace Model = Aws::S3::Model; + +struct ClientCache +{ + ClientCache() = default; + + ClientCache(const ClientCache & other) + : region_for_bucket_cache(other.region_for_bucket_cache) + , uri_for_bucket_cache(other.uri_for_bucket_cache) + {} + + ClientCache(ClientCache && other) = delete; + + ClientCache & operator=(const ClientCache &) = delete; + ClientCache & operator=(ClientCache &&) = delete; + + void clearCache(); + + std::mutex region_cache_mutex; + std::unordered_map region_for_bucket_cache; + + std::mutex uri_cache_mutex; + std::unordered_map uri_for_bucket_cache; +}; + +class ClientCacheRegistry +{ +public: + static ClientCacheRegistry & instance() + { + static ClientCacheRegistry registry; + return registry; + } + + void registerClient(const std::shared_ptr & client_cache); + void unregisterClient(ClientCache * client); + void clearCacheForAll(); +private: + ClientCacheRegistry() = default; + + std::mutex clients_mutex; + std::unordered_map> client_caches; +}; + +/// Client that improves the client from the AWS SDK +/// - inject region and URI into requests so they are rerouted to the correct destination if needed +/// - automatically detect endpoint and regions for each bucket and cache them +/// +/// For this client to work correctly both Client::RetryStrategy and Requests defined in should be used. +class Client : public Aws::S3::S3Client +{ +public: + template + static std::unique_ptr create(Args &&... args) + { + (verifyArgument(args), ...); + return std::unique_ptr(new Client(std::forward(args)...)); + } + + Client & operator=(const Client &) = delete; + + Client(Client && other) = delete; + Client & operator=(Client &&) = delete; + + ~Client() override + { + try + { + ClientCacheRegistry::instance().unregisterClient(cache.get()); + } + catch (...) + { + tryLogCurrentException(log); + throw; + } + } + + /// Decorator for RetryStrategy needed for this client to work correctly + class RetryStrategy : public Aws::Client::RetryStrategy + { + public: + explicit RetryStrategy(std::shared_ptr wrapped_strategy_); + + /// NOLINTNEXTLINE(google-runtime-int) + bool ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; + + /// NOLINTNEXTLINE(google-runtime-int) + long CalculateDelayBeforeNextRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; + + /// NOLINTNEXTLINE(google-runtime-int) + long GetMaxAttempts() const override; + + void GetSendToken() override; + + bool HasSendToken() override; + + void RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome) override; + void RequestBookkeeping(const Aws::Client::HttpResponseOutcome& httpResponseOutcome, const Aws::Client::AWSError& lastError) override; + private: + std::shared_ptr wrapped_strategy; + }; + + Model::HeadObjectOutcome HeadObject(const HeadObjectRequest & request) const; + Model::ListObjectsV2Outcome ListObjectsV2(const ListObjectsV2Request & request) const; + Model::ListObjectsOutcome ListObjects(const ListObjectsRequest & request) const; + Model::GetObjectOutcome GetObject(const GetObjectRequest & request) const; + + Model::AbortMultipartUploadOutcome AbortMultipartUpload(const AbortMultipartUploadRequest & request) const; + Model::CreateMultipartUploadOutcome CreateMultipartUpload(const CreateMultipartUploadRequest & request) const; + Model::CompleteMultipartUploadOutcome CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const; + Model::UploadPartOutcome UploadPart(const UploadPartRequest & request) const; + Model::UploadPartCopyOutcome UploadPartCopy(const UploadPartCopyRequest & request) const; + + Model::CopyObjectOutcome CopyObject(const CopyObjectRequest & request) const; + Model::PutObjectOutcome PutObject(const PutObjectRequest & request) const; + Model::DeleteObjectOutcome DeleteObject(const DeleteObjectRequest & request) const; + Model::DeleteObjectsOutcome DeleteObjects(const DeleteObjectsRequest & request) const; + +private: + template + explicit Client(size_t max_redirects_, Args &&... args) + : Aws::S3::S3Client(std::forward(args)...) + , max_redirects(max_redirects_) + , log(&Poco::Logger::get("S3Client")) + { + auto * endpoint_provider = dynamic_cast(accessEndpointProvider().get()); + endpoint_provider->GetBuiltInParameters().GetParameter("Region").GetString(explicit_region); + std::string endpoint; + endpoint_provider->GetBuiltInParameters().GetParameter("Endpoint").GetString(endpoint); + detect_region = explicit_region == Aws::Region::AWS_GLOBAL && endpoint.find(".amazonaws.com") != std::string::npos; + + cache = std::make_shared(); + ClientCacheRegistry::instance().registerClient(cache); + } + + Client(const Client & other) + : Aws::S3::S3Client(other) + , explicit_region(other.explicit_region) + , detect_region(other.detect_region) + , max_redirects(other.max_redirects) + , log(&Poco::Logger::get("S3Client")) + { + cache = std::make_shared(*other.cache); + ClientCacheRegistry::instance().registerClient(cache); + } + + /// Make regular functions private + using Aws::S3::S3Client::HeadObject; + using Aws::S3::S3Client::ListObjectsV2; + using Aws::S3::S3Client::ListObjects; + using Aws::S3::S3Client::GetObject; + + using Aws::S3::S3Client::AbortMultipartUpload; + using Aws::S3::S3Client::CreateMultipartUpload; + using Aws::S3::S3Client::CompleteMultipartUpload; + using Aws::S3::S3Client::UploadPart; + using Aws::S3::S3Client::UploadPartCopy; + + using Aws::S3::S3Client::CopyObject; + using Aws::S3::S3Client::PutObject; + using Aws::S3::S3Client::DeleteObject; + using Aws::S3::S3Client::DeleteObjects; + + template + std::invoke_result_t + doRequest(const RequestType & request, RequestFn request_fn) const + { + const auto & bucket = request.GetBucket(); + + if (auto region = getRegionForBucket(bucket); !region.empty()) + { + if (!detect_region) + LOG_INFO(log, "Using region override {} for bucket {}", region, bucket); + + request.overrideRegion(std::move(region)); + } + + if (auto uri = getURIForBucket(bucket); uri.has_value()) + request.overrideURI(std::move(*uri)); + + + bool found_new_endpoint = false; + // if we found correct endpoint after 301 responses, update the cache for future requests + SCOPE_EXIT( + if (found_new_endpoint) + { + auto uri_override = request.getURIOverride(); + assert(uri_override.has_value()); + updateURIForBucket(bucket, std::move(*uri_override)); + } + ); + + for (size_t attempt = 0; attempt <= max_redirects; ++attempt) + { + auto result = request_fn(request); + if (result.IsSuccess()) + return result; + + const auto & error = result.GetError(); + + std::string new_region; + if (checkIfWrongRegionDefined(bucket, error, new_region)) + { + request.overrideRegion(new_region); + continue; + } + + if (error.GetResponseCode() != Aws::Http::HttpResponseCode::MOVED_PERMANENTLY) + return result; + + // maybe we detect a correct region + if (!detect_region) + { + if (auto region = GetErrorMarshaller()->ExtractRegion(error); !region.empty() && region != explicit_region) + { + request.overrideRegion(region); + insertRegionOverride(bucket, region); + } + } + + // we possibly got new location, need to try with that one + auto new_uri = getURIFromError(error); + if (!new_uri) + return result; + + const auto & current_uri_override = request.getURIOverride(); + /// we already tried with this URI + if (current_uri_override && current_uri_override->uri == new_uri->uri) + { + LOG_INFO(log, "Getting redirected to the same invalid location {}", new_uri->uri.toString()); + return result; + } + + found_new_endpoint = true; + request.overrideURI(*new_uri); + } + + throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects"); + } + + void updateURIForBucket(const std::string & bucket, S3::URI new_uri) const; + std::optional getURIFromError(const Aws::S3::S3Error & error) const; + std::optional updateURIForBucketForHead(const std::string & bucket) const; + + std::string getRegionForBucket(const std::string & bucket, bool force_detect = false) const; + std::optional getURIForBucket(const std::string & bucket) const; + + bool checkIfWrongRegionDefined(const std::string & bucket, const Aws::S3::S3Error & error, std::string & region) const; + void insertRegionOverride(const std::string & bucket, const std::string & region) const; + + template + static void verifyArgument(const T & /*arg*/) + {} + + template T> + static void verifyArgument(const T & client_config) + { + if (!client_config.retryStrategy) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The S3 client can only be used with Client::RetryStrategy, define it in the client configuration"); + + assert_cast(*client_config.retryStrategy); + } + + std::string explicit_region; + mutable bool detect_region = true; + + mutable std::shared_ptr cache; + + const size_t max_redirects; + + Poco::Logger * log; +}; + +} + +} + +#endif diff --git a/src/IO/S3/Requests.h b/src/IO/S3/Requests.h new file mode 100644 index 00000000000..fac1a3eff0a --- /dev/null +++ b/src/IO/S3/Requests.h @@ -0,0 +1,87 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::S3 +{ + +namespace Model = Aws::S3::Model; + +template +class ExtendedRequest : public BaseRequest +{ +public: + Aws::Endpoint::EndpointParameters GetEndpointContextParams() const override + { + auto params = BaseRequest::GetEndpointContextParams(); + if (!region_override.empty()) + params.emplace_back("Region", region_override); + + if (uri_override.has_value()) + { + static const Aws::String AWS_S3_FORCE_PATH_STYLE = "ForcePathStyle"; + params.emplace_back(AWS_S3_FORCE_PATH_STYLE, !uri_override->is_virtual_hosted_style); + params.emplace_back("Endpoint", uri_override->endpoint); + } + + return params; + } + + void overrideRegion(std::string region) const + { + region_override = std::move(region); + } + + void overrideURI(S3::URI uri) const + { + uri_override = std::move(uri); + } + + const auto & getURIOverride() const + { + return uri_override; + } + +protected: + mutable std::string region_override; + mutable std::optional uri_override; +}; + +using HeadObjectRequest = ExtendedRequest; +using ListObjectsV2Request = ExtendedRequest; +using ListObjectsRequest = ExtendedRequest; +using GetObjectRequest = ExtendedRequest; + +using CreateMultipartUploadRequest = ExtendedRequest; +using CompleteMultipartUploadRequest = ExtendedRequest; +using AbortMultipartUploadRequest = ExtendedRequest; +using UploadPartRequest = ExtendedRequest; +using UploadPartCopyRequest = ExtendedRequest; + +using PutObjectRequest = ExtendedRequest; +using CopyObjectRequest = ExtendedRequest; +using DeleteObjectRequest = ExtendedRequest; +using DeleteObjectsRequest = ExtendedRequest; + +} + +#endif diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp new file mode 100644 index 00000000000..9d3496377ce --- /dev/null +++ b/src/IO/S3/URI.cpp @@ -0,0 +1,119 @@ +#include + +#if USE_AWS_S3 +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace S3 +{ + +URI::URI(const std::string & uri_) +{ + /// Case when bucket name represented in domain name of S3 URL. + /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) + /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access + static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos|obs|oss)([.\-][a-z0-9\-.:]+))"); + + /// Case when bucket name and key represented in path of S3 URL. + /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) + /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#path-style-access + static const RE2 path_style_pattern("^/([^/]*)/(.*)"); + + static constexpr auto S3 = "S3"; + static constexpr auto COSN = "COSN"; + static constexpr auto COS = "COS"; + static constexpr auto OBS = "OBS"; + static constexpr auto OSS = "OSS"; + + uri = Poco::URI(uri_); + + storage_name = S3; + + if (uri.getHost().empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Host is empty in S3 URI."); + + /// Extract object version ID from query string. + bool has_version_id = false; + for (const auto & [query_key, query_value] : uri.getQueryParameters()) + if (query_key == "versionId") + { + version_id = query_value; + has_version_id = true; + } + + /// Poco::URI will ignore '?' when parsing the path, but if there is a vestionId in the http parameter, + /// '?' can not be used as a wildcard, otherwise it will be ambiguous. + /// If no "vertionId" in the http parameter, '?' can be used as a wildcard. + /// It is necessary to encode '?' to avoid deletion during parsing path. + if (!has_version_id && uri_.find('?') != String::npos) + { + String uri_with_question_mark_encode; + Poco::URI::encode(uri_, "?", uri_with_question_mark_encode); + uri = Poco::URI(uri_with_question_mark_encode); + } + + String name; + String endpoint_authority_from_uri; + + if (re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) + { + is_virtual_hosted_style = true; + endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri; + validateBucket(bucket, uri); + + if (!uri.getPath().empty()) + { + /// Remove leading '/' from path to extract key. + key = uri.getPath().substr(1); + } + + boost::to_upper(name); + if (name != S3 && name != COS && name != OBS && name != OSS) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", + quoteString(name)); + + if (name == S3) + storage_name = name; + else if (name == OBS) + storage_name = OBS; + else if (name == OSS) + storage_name = OSS; + else + storage_name = COSN; + } + else if (re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) + { + is_virtual_hosted_style = false; + endpoint = uri.getScheme() + "://" + uri.getAuthority(); + validateBucket(bucket, uri); + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket or key name are invalid in S3 URI."); +} + +void URI::validateBucket(const String & bucket, const Poco::URI & uri) +{ + /// S3 specification requires at least 3 and at most 63 characters in bucket name. + /// https://docs.aws.amazon.com/awscloudtrail/latest/userguide/cloudtrail-s3-bucket-naming-requirements.html + if (bucket.length() < 3 || bucket.length() > 63) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket name length is out of bounds in virtual hosted style S3 URI: {}{}", + quoteString(bucket), !uri.empty() ? " (" + uri.toString() + ")" : ""); +} + +} + +} + +#endif diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h new file mode 100644 index 00000000000..f21d30929ee --- /dev/null +++ b/src/IO/S3/URI.h @@ -0,0 +1,40 @@ +#pragma once + +#include + +#include "config.h" + +#if USE_AWS_S3 + +#include + +namespace DB::S3 +{ + +/** + * Represents S3 URI. + * + * The following patterns are allowed: + * s3://bucket/key + * http(s)://endpoint/bucket/key + */ +struct URI +{ + Poco::URI uri; + // Custom endpoint if URI scheme is not S3. + std::string endpoint; + std::string bucket; + std::string key; + std::string version_id; + std::string storage_name; + + bool is_virtual_hosted_style; + + explicit URI(const std::string & uri_); + + static void validateBucket(const std::string & bucket, const Poco::URI & uri); +}; + +} + +#endif diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index c4f9718e906..da8ea0c0481 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -9,15 +9,7 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include - +#include namespace ProfileEvents { @@ -54,7 +46,7 @@ namespace { public: UploadHelper( - const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -77,7 +69,7 @@ namespace virtual ~UploadHelper() = default; protected: - std::shared_ptr client_ptr; + std::shared_ptr client_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -108,7 +100,7 @@ namespace void createMultipartUpload() { - Aws::S3::Model::CreateMultipartUploadRequest request; + S3::CreateMultipartUploadRequest request; request.SetBucket(dest_bucket); request.SetKey(dest_key); @@ -147,7 +139,7 @@ namespace if (part_tags.empty()) throw Exception(ErrorCodes::S3_ERROR, "Failed to complete multipart upload. No parts have uploaded"); - Aws::S3::Model::CompleteMultipartUploadRequest request; + S3::CompleteMultipartUploadRequest request; request.SetBucket(dest_bucket); request.SetKey(dest_key); request.SetUploadId(multipart_upload_id); @@ -194,7 +186,7 @@ namespace void abortMultipartUpload() { LOG_TRACE(log, "Aborting multipart upload. Bucket: {}, Key: {}, Upload_id: {}", dest_bucket, dest_key, multipart_upload_id); - Aws::S3::Model::AbortMultipartUploadRequest abort_request; + S3::AbortMultipartUploadRequest abort_request; abort_request.SetBucket(dest_bucket); abort_request.SetKey(dest_key); abort_request.SetUploadId(multipart_upload_id); @@ -404,7 +396,7 @@ namespace const std::function()> & create_read_buffer_, size_t offset_, size_t size_, - const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -436,12 +428,12 @@ namespace void performSinglepartUpload() { - Aws::S3::Model::PutObjectRequest request; + S3::PutObjectRequest request; fillPutRequest(request); processPutRequest(request); } - void fillPutRequest(Aws::S3::Model::PutObjectRequest & request) + void fillPutRequest(S3::PutObjectRequest & request) { auto read_buffer = std::make_unique(create_read_buffer(), offset, size); @@ -461,7 +453,7 @@ namespace request.SetContentType("binary/octet-stream"); } - void processPutRequest(const Aws::S3::Model::PutObjectRequest & request) + void processPutRequest(const S3::PutObjectRequest & request) { size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t retries = 1;; ++retries) @@ -526,7 +518,7 @@ namespace auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); /// Setup request. - auto request = std::make_unique(); + auto request = std::make_unique(); request->SetBucket(dest_bucket); request->SetKey(dest_key); request->SetPartNumber(static_cast(part_number)); @@ -542,7 +534,7 @@ namespace String processUploadPartRequest(Aws::AmazonWebServiceRequest & request) override { - auto & req = typeid_cast(request); + auto & req = typeid_cast(request); ProfileEvents::increment(ProfileEvents::S3UploadPart); if (for_disk_s3) @@ -564,7 +556,7 @@ namespace { public: CopyFileHelper( - const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -602,12 +594,12 @@ namespace void performSingleOperationCopy() { - Aws::S3::Model::CopyObjectRequest request; + S3::CopyObjectRequest request; fillCopyRequest(request); processCopyRequest(request); } - void fillCopyRequest(Aws::S3::Model::CopyObjectRequest & request) + void fillCopyRequest(S3::CopyObjectRequest & request) { request.SetCopySource(src_bucket + "/" + src_key); request.SetBucket(dest_bucket); @@ -627,7 +619,7 @@ namespace request.SetContentType("binary/octet-stream"); } - void processCopyRequest(const Aws::S3::Model::CopyObjectRequest & request) + void processCopyRequest(const S3::CopyObjectRequest & request) { size_t max_retries = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t retries = 1;; ++retries) @@ -689,7 +681,7 @@ namespace std::unique_ptr fillUploadPartRequest(size_t part_number, size_t part_offset, size_t part_size) override { - auto request = std::make_unique(); + auto request = std::make_unique(); /// Make a copy request to copy a part. request->SetCopySource(src_bucket + "/" + src_key); @@ -704,7 +696,7 @@ namespace String processUploadPartRequest(Aws::AmazonWebServiceRequest & request) override { - auto & req = typeid_cast(request); + auto & req = typeid_cast(request); ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); if (for_disk_s3) @@ -727,7 +719,7 @@ void copyDataToS3File( const std::function()> & create_read_buffer, size_t offset, size_t size, - const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -741,7 +733,7 @@ void copyDataToS3File( void copyS3File( - const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, size_t src_offset, diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 808860650b8..5204af5aa4b 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -21,7 +20,7 @@ class SeekableReadBuffer; /// however copyS3File() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. void copyS3File( - const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, size_t src_offset, @@ -42,7 +41,7 @@ void copyDataToS3File( const std::function()> & create_read_buffer, size_t offset, size_t size, - const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index 04871682296..20d5e74d6d4 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -1,11 +1,6 @@ #include #if USE_AWS_S3 -#include -#include -#include -#include - namespace ErrorCodes { @@ -30,13 +25,13 @@ namespace DB::S3 namespace { Aws::S3::Model::HeadObjectOutcome headObject( - const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) + const S3::Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) { ProfileEvents::increment(ProfileEvents::S3HeadObject); if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3HeadObject); - Aws::S3::Model::HeadObjectRequest req; + S3::HeadObjectRequest req; req.SetBucket(bucket); req.SetKey(key); @@ -46,93 +41,25 @@ namespace return client.HeadObject(req); } - Aws::S3::Model::GetObjectAttributesOutcome getObjectAttributes(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) - { - ProfileEvents::increment(ProfileEvents::S3GetObjectAttributes); - if (for_disk_s3) - ProfileEvents::increment(ProfileEvents::DiskS3GetObjectAttributes); - - Aws::S3::Model::GetObjectAttributesRequest req; - req.SetBucket(bucket); - req.SetKey(key); - - if (!version_id.empty()) - req.SetVersionId(version_id); - - req.SetObjectAttributes({Aws::S3::Model::ObjectAttributes::ObjectSize}); - - return client.GetObjectAttributes(req); - } - - Aws::S3::Model::GetObjectOutcome getObjectDummy(const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) - { - ProfileEvents::increment(ProfileEvents::S3GetObject); - if (for_disk_s3) - ProfileEvents::increment(ProfileEvents::DiskS3GetObject); - - Aws::S3::Model::GetObjectRequest req; - req.SetBucket(bucket); - req.SetKey(key); - - if (!version_id.empty()) - req.SetVersionId(version_id); - - /// Only the first byte will be read. - /// We don't need that first byte but the range should be set otherwise the entire object will be read. - req.SetRange("bytes=0-0"); - - return client.GetObject(req); - } - - /// Performs a request to get the size and last modification time of an object. /// The function performs either HeadObject or GetObjectAttributes request depending on the endpoint. std::pair, Aws::S3::S3Error> tryGetObjectInfo( - const Aws::S3::S3Client & client, const String & bucket, const String & key, const String & version_id, - const S3Settings::RequestSettings & request_settings, bool with_metadata, bool for_disk_s3) + const S3::Client & client, const String & bucket, const String & key, const String & version_id, + const S3Settings::RequestSettings & /*request_settings*/, bool with_metadata, bool for_disk_s3) { - if (request_settings.allow_head_object_request) - { - auto outcome = headObject(client, bucket, key, version_id, for_disk_s3); - if (!outcome.IsSuccess()) - return {std::nullopt, outcome.GetError()}; + auto outcome = headObject(client, bucket, key, version_id, for_disk_s3); + if (!outcome.IsSuccess()) + return {std::nullopt, outcome.GetError()}; - const auto & result = outcome.GetResult(); - ObjectInfo object_info; - object_info.size = static_cast(result.GetContentLength()); - object_info.last_modification_time = result.GetLastModified().Millis() / 1000; + const auto & result = outcome.GetResult(); + ObjectInfo object_info; + object_info.size = static_cast(result.GetContentLength()); + object_info.last_modification_time = result.GetLastModified().Millis() / 1000; - if (with_metadata) - object_info.metadata = result.GetMetadata(); + if (with_metadata) + object_info.metadata = result.GetMetadata(); - return {object_info, {}}; - } - else - { - ObjectInfo object_info; - - { - auto outcome = getObjectAttributes(client, bucket, key, version_id, for_disk_s3); - if (!outcome.IsSuccess()) - return {std::nullopt, outcome.GetError()}; - - const auto & result = outcome.GetResult(); - object_info.size = static_cast(result.GetObjectSize()); - object_info.last_modification_time = result.GetLastModified().Millis() / 1000; - } - - if (with_metadata) - { - auto outcome = getObjectDummy(client, bucket, key, version_id, for_disk_s3); - if (!outcome.IsSuccess()) - return {std::nullopt, outcome.GetError()}; - - const auto & result = outcome.GetResult(); - object_info.metadata = result.GetMetadata(); - } - - return {object_info, {}}; - } + return {object_info, {}}; } } @@ -143,7 +70,7 @@ bool isNotFoundError(Aws::S3::S3Errors error) } ObjectInfo getObjectInfo( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id, @@ -167,7 +94,7 @@ ObjectInfo getObjectInfo( } size_t getObjectSize( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id, @@ -179,7 +106,7 @@ size_t getObjectSize( } bool objectExists( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id, @@ -199,7 +126,7 @@ bool objectExists( } void checkObjectExists( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id, diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index 363e0601f32..06c7d386e43 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -5,7 +5,7 @@ #if USE_AWS_S3 #include #include -#include +#include namespace DB::S3 @@ -20,7 +20,7 @@ struct ObjectInfo }; ObjectInfo getObjectInfo( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id = {}, @@ -30,7 +30,7 @@ ObjectInfo getObjectInfo( bool throw_on_error = true); size_t getObjectSize( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id = {}, @@ -39,7 +39,7 @@ size_t getObjectSize( bool throw_on_error = true); bool objectExists( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id = {}, @@ -48,7 +48,7 @@ bool objectExists( /// Throws an exception if a specified object doesn't exist. `description` is used as a part of the error message. void checkObjectExists( - const Aws::S3::S3Client & client, + const S3::Client & client, const String & bucket, const String & key, const String & version_id = {}, diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index da1a43a1dec..bbbf6a430ab 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -19,13 +19,13 @@ #include #include #include -#include #include #include #include #include #include +#include #include #include @@ -102,7 +102,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) bool use_environment_credentials = false; bool use_insecure_imds_request = false; - std::shared_ptr client = DB::S3::ClientFactory::instance().create( + std::shared_ptr client = DB::S3::ClientFactory::instance().create( client_configuration, uri.is_virtual_hosted_style, access_key_id, diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index e590cbdcf31..50fee97870d 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -16,7 +16,6 @@ # include # include # include -# include # include # include # include @@ -26,16 +25,12 @@ # include # include # include -# include -# include -# include -# include # include # include -# include -# include -# include +# include +# include +# include # include # include @@ -712,7 +707,6 @@ namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } @@ -738,7 +732,7 @@ namespace S3 return ret; } - std::unique_ptr ClientFactory::create( // NOLINT + std::unique_ptr ClientFactory::create( // NOLINT const PocoHTTPClientConfiguration & cfg_, bool is_virtual_hosted_style, const String & access_key_id, @@ -753,7 +747,7 @@ namespace S3 if (!server_side_encryption_customer_key_base64.empty()) { - /// See S3Client::GeneratePresignedUrlWithSSEC(). + /// See Client::GeneratePresignedUrlWithSSEC(). headers.push_back({Aws::S3::SSEHeaders::SERVER_SIDE_ENCRYPTION_CUSTOMER_ALGORITHM, Aws::S3::Model::ServerSideEncryptionMapper::GetNameForServerSideEncryption(Aws::S3::Model::ServerSideEncryption::AES256)}); @@ -776,7 +770,9 @@ namespace S3 use_environment_credentials, use_insecure_imds_request); - return std::make_unique( + client_configuration.retryStrategy = std::make_shared(std::move(client_configuration.retryStrategy)); + return Client::create( + client_configuration.s3_max_redirects, std::move(credentials_provider), std::move(client_configuration), // Client configuration. Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, @@ -802,100 +798,6 @@ namespace S3 get_request_throttler, put_request_throttler); } - - URI::URI(const std::string & uri_) - { - /// Case when bucket name represented in domain name of S3 URL. - /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) - /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access - static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos|obs|oss)([.\-][a-z0-9\-.:]+))"); - - /// Case when bucket name and key represented in path of S3 URL. - /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) - /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#path-style-access - static const RE2 path_style_pattern("^/([^/]*)/(.*)"); - - static constexpr auto S3 = "S3"; - static constexpr auto COSN = "COSN"; - static constexpr auto COS = "COS"; - static constexpr auto OBS = "OBS"; - static constexpr auto OSS = "OSS"; - - uri = Poco::URI(uri_); - - storage_name = S3; - - if (uri.getHost().empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Host is empty in S3 URI."); - - /// Extract object version ID from query string. - bool has_version_id = false; - for (const auto & [query_key, query_value] : uri.getQueryParameters()) - if (query_key == "versionId") - { - version_id = query_value; - has_version_id = true; - } - - /// Poco::URI will ignore '?' when parsing the path, but if there is a vestionId in the http parameter, - /// '?' can not be used as a wildcard, otherwise it will be ambiguous. - /// If no "vertionId" in the http parameter, '?' can be used as a wildcard. - /// It is necessary to encode '?' to avoid deletion during parsing path. - if (!has_version_id && uri_.find('?') != String::npos) - { - String uri_with_question_mark_encode; - Poco::URI::encode(uri_, "?", uri_with_question_mark_encode); - uri = Poco::URI(uri_with_question_mark_encode); - } - - String name; - String endpoint_authority_from_uri; - - if (re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) - { - is_virtual_hosted_style = true; - endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri; - validateBucket(bucket, uri); - - if (!uri.getPath().empty()) - { - /// Remove leading '/' from path to extract key. - key = uri.getPath().substr(1); - } - - boost::to_upper(name); - if (name != S3 && name != COS && name != OBS && name != OSS) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", - quoteString(name)); - - if (name == S3) - storage_name = name; - else if (name == OBS) - storage_name = OBS; - else if (name == OSS) - storage_name = OSS; - else - storage_name = COSN; - } - else if (re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) - { - is_virtual_hosted_style = false; - endpoint = uri.getScheme() + "://" + uri.getAuthority(); - validateBucket(bucket, uri); - } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket or key name are invalid in S3 URI."); - } - - void URI::validateBucket(const String & bucket, const Poco::URI & uri) - { - /// S3 specification requires at least 3 and at most 63 characters in bucket name. - /// https://docs.aws.amazon.com/awscloudtrail/latest/userguide/cloudtrail-s3-bucket-naming-requirements.html - if (bucket.length() < 3 || bucket.length() > 63) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bucket name length is out of bounds in virtual hosted style S3 URI: {}{}", - quoteString(bucket), !uri.empty() ? " (" + uri.toString() + ")" : ""); - } } } diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 19e660a338d..8bf0dd7e846 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -14,12 +14,13 @@ #include #include -#include +#include +#include + #include #include - -namespace Aws::S3 { class S3Client; } +namespace Aws::S3 { class Client; } namespace DB { @@ -60,7 +61,6 @@ private: }; } - namespace DB::S3 { @@ -71,7 +71,7 @@ public: static ClientFactory & instance(); - std::unique_ptr create( + std::unique_ptr create( const PocoHTTPClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, @@ -97,30 +97,6 @@ private: std::atomic s3_requests_logging_enabled; }; -/** - * Represents S3 URI. - * - * The following patterns are allowed: - * s3://bucket/key - * http(s)://endpoint/bucket/key - */ -struct URI -{ - Poco::URI uri; - // Custom endpoint if URI scheme is not S3. - String endpoint; - String bucket; - String key; - String version_id; - String storage_name; - - bool is_virtual_hosted_style; - - explicit URI(const std::string & uri_); - - static void validateBucket(const String & bucket, const Poco::URI & uri); -}; - } #endif diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c71a0e6a252..c90167379d5 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -9,15 +9,11 @@ #include #include +#include +#include #include #include -#include -#include -#include -#include -#include -#include #include #include @@ -28,13 +24,11 @@ namespace ProfileEvents extern const Event WriteBufferFromS3Bytes; extern const Event S3WriteBytes; - extern const Event S3HeadObject; extern const Event S3CreateMultipartUpload; extern const Event S3CompleteMultipartUpload; extern const Event S3UploadPart; extern const Event S3PutObject; - extern const Event DiskS3HeadObject; extern const Event DiskS3CreateMultipartUpload; extern const Event DiskS3CompleteMultipartUpload; extern const Event DiskS3UploadPart; @@ -59,7 +53,7 @@ namespace ErrorCodes struct WriteBufferFromS3::UploadPartTask { - Aws::S3::Model::UploadPartRequest req; + S3::UploadPartRequest req; bool is_finished = false; std::string tag; std::exception_ptr exception; @@ -67,13 +61,13 @@ struct WriteBufferFromS3::UploadPartTask struct WriteBufferFromS3::PutObjectTask { - Aws::S3::Model::PutObjectRequest req; + S3::PutObjectRequest req; bool is_finished = false; std::exception_ptr exception; }; WriteBufferFromS3::WriteBufferFromS3( - std::shared_ptr client_ptr_, + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, const S3Settings::RequestSettings & request_settings_, @@ -191,7 +185,7 @@ void WriteBufferFromS3::finalizeImpl() void WriteBufferFromS3::createMultipartUpload() { - Aws::S3::Model::CreateMultipartUploadRequest req; + DB::S3::CreateMultipartUploadRequest req; req.SetBucket(bucket); req.SetKey(key); @@ -298,7 +292,7 @@ void WriteBufferFromS3::writePart() } } -void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & req) +void WriteBufferFromS3::fillUploadRequest(S3::UploadPartRequest & req) { /// Increase part number. ++part_number; @@ -369,7 +363,7 @@ void WriteBufferFromS3::completeMultipartUpload() if (tags.empty()) throw Exception(ErrorCodes::S3_ERROR, "Failed to complete multipart upload. No parts have uploaded"); - Aws::S3::Model::CompleteMultipartUploadRequest req; + S3::CompleteMultipartUploadRequest req; req.SetBucket(bucket); req.SetKey(key); req.SetUploadId(multipart_upload_id); @@ -474,7 +468,7 @@ void WriteBufferFromS3::makeSinglepartUpload() } } -void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) +void WriteBufferFromS3::fillPutRequest(S3::PutObjectRequest & req) { req.SetBucket(bucket); req.SetKey(key); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 8c81c49d32c..e56d590c57a 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -22,13 +23,7 @@ namespace Aws::S3 { -class S3Client; -} - -namespace Aws::S3::Model -{ - class UploadPartRequest; - class PutObjectRequest; +class Client; } namespace DB @@ -47,7 +42,7 @@ class WriteBufferFromS3 final : public BufferWithOwnMemory { public: WriteBufferFromS3( - std::shared_ptr client_ptr_, + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, const S3Settings::RequestSettings & request_settings_, @@ -75,11 +70,11 @@ private: void finalizeImpl() override; struct UploadPartTask; - void fillUploadRequest(Aws::S3::Model::UploadPartRequest & req); + void fillUploadRequest(S3::UploadPartRequest & req); void processUploadRequest(UploadPartTask & task); struct PutObjectTask; - void fillPutRequest(Aws::S3::Model::PutObjectRequest & req); + void fillPutRequest(S3::PutObjectRequest & req); void processPutRequest(const PutObjectTask & task); void waitForReadyBackGroundTasks(); @@ -90,7 +85,7 @@ private: const String key; const S3Settings::RequestSettings request_settings; const S3Settings::RequestSettings::PartUploadSettings & upload_settings; - const std::shared_ptr client_ptr; + const std::shared_ptr client_ptr; const std::optional> object_metadata; size_t upload_part_size = 0; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 6d4722dbf20..9a2252844d1 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -58,6 +58,10 @@ #include #include +#if USE_AWS_S3 +#include +#endif + #include "config.h" namespace DB @@ -338,6 +342,13 @@ BlockIO InterpreterSystemQuery::execute() cache->reset(); break; #endif +#if USE_AWS_S3 + case Type::DROP_S3_CLIENT_CACHE: + getContext()->checkAccess(AccessType::SYSTEM_DROP_S3_CLIENT_CACHE); + S3::ClientCacheRegistry::instance().clearCacheForAll(); + break; +#endif + case Type::DROP_FILESYSTEM_CACHE: { getContext()->checkAccess(AccessType::SYSTEM_DROP_FILESYSTEM_CACHE); @@ -978,6 +989,9 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_INDEX_UNCOMPRESSED_CACHE: case Type::DROP_FILESYSTEM_CACHE: case Type::DROP_SCHEMA_CACHE: +#if USE_AWS_S3 + case Type::DROP_S3_CLIENT_CACHE: +#endif { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 02ddbc7dcd2..80a891712a6 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -31,6 +31,9 @@ public: #endif DROP_FILESYSTEM_CACHE, DROP_SCHEMA_CACHE, +#if USE_AWS_S3 + DROP_S3_CLIENT_CACHE, +#endif STOP_LISTEN_QUERIES, START_LISTEN_QUERIES, RESTART_REPLICAS, diff --git a/src/Storages/StorageDeltaLake.cpp b/src/Storages/StorageDeltaLake.cpp index 479a11b5eb4..c74e37a207c 100644 --- a/src/Storages/StorageDeltaLake.cpp +++ b/src/Storages/StorageDeltaLake.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -16,8 +17,6 @@ #include #include -#include -#include #include @@ -101,7 +100,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() const auto & client = base_configuration.client; - Aws::S3::Model::ListObjectsV2Request request; + S3::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; bool is_finished{false}; diff --git a/src/Storages/StorageDeltaLake.h b/src/Storages/StorageDeltaLake.h index af6485b9a40..5e3fdd8ad71 100644 --- a/src/Storages/StorageDeltaLake.h +++ b/src/Storages/StorageDeltaLake.h @@ -15,11 +15,6 @@ namespace Poco class Logger; } -namespace Aws::S3 -{ -class S3Client; -} - namespace DB { diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index d5675ceb17c..1b0de3c3ed2 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -7,12 +7,11 @@ #include #include +#include #include #include #include #include -#include -#include #include @@ -98,7 +97,7 @@ std::vector getKeysFromS3(const StorageS3::S3Configuration & base_c const auto & client = base_configuration.client; - Aws::S3::Model::ListObjectsV2Request request; + S3::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; bool is_finished{false}; diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 00b8c01a46d..d4a9aa7ff09 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -12,11 +12,6 @@ namespace Poco class Logger; } -namespace Aws::S3 -{ -class S3Client; -} - namespace DB { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 205b0c7d067..d7ea64c50d6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include @@ -51,10 +52,6 @@ #include #include -#include -#include -#include -#include #include #include @@ -136,7 +133,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext { public: Impl( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const S3::URI & globbed_uri_, ASTPtr & query_, const Block & virtual_header_, @@ -145,7 +142,7 @@ public: Strings * read_keys_, const S3Settings::RequestSettings & request_settings_) : WithContext(context_) - , client(client_) + , client(S3::Client::create(client_)) , globbed_uri(globbed_uri_) , query(query_) , virtual_header(virtual_header_) @@ -349,7 +346,7 @@ private: return list_objects_scheduler([this] { ProfileEvents::increment(ProfileEvents::S3ListObjects); - auto outcome = client.ListObjectsV2(request); + auto outcome = client->ListObjectsV2(request); /// Outcome failure will be handled on the caller side. if (outcome.IsSuccess()) @@ -364,7 +361,7 @@ private: KeysWithInfo buffer; KeysWithInfo::iterator buffer_iter; - Aws::S3::S3Client client; + std::unique_ptr client; S3::URI globbed_uri; ASTPtr query; Block virtual_header; @@ -376,7 +373,7 @@ private: ObjectInfos * object_infos; Strings * read_keys; - Aws::S3::Model::ListObjectsV2Request request; + S3::ListObjectsV2Request request; S3Settings::RequestSettings request_settings; ThreadPool list_objects_pool; @@ -386,7 +383,7 @@ private: }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const S3::URI & globbed_uri_, ASTPtr query, const Block & virtual_header, @@ -412,7 +409,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext { public: explicit Impl( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const std::string & version_id_, const std::vector & keys_, const String & bucket_, @@ -507,7 +504,7 @@ private: }; StorageS3Source::KeysIterator::KeysIterator( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const std::string & version_id_, const std::vector & keys_, const String & bucket_, @@ -552,7 +549,7 @@ StorageS3Source::StorageS3Source( UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, String compression_hint_, - const std::shared_ptr & client_, + const std::shared_ptr & client_, const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, @@ -1201,7 +1198,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, } ProfileEvents::increment(ProfileEvents::S3DeleteObjects); - Aws::S3::Model::DeleteObjectsRequest request; + S3::DeleteObjectsRequest request; request.SetBucket(s3_configuration.uri.bucket); request.SetDelete(delkeys); @@ -1211,6 +1208,9 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, const auto & err = response.GetError(); throw Exception(ErrorCodes::S3_ERROR, "{}: {}", std::to_string(static_cast(err.GetErrorType())), err.GetMessage()); } + + for (const auto & error : response.GetResult().GetErrors()) + LOG_WARNING(&Poco::Logger::get("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage()); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 16e38249595..65c6928906d 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -24,7 +24,7 @@ namespace Aws::S3 { - class S3Client; + class Client; } namespace DB @@ -66,7 +66,7 @@ public: { public: DisclosedGlobIterator( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const S3::URI & globbed_uri_, ASTPtr query, const Block & virtual_header, @@ -88,7 +88,7 @@ public: { public: explicit KeysIterator( - const Aws::S3::S3Client & client_, + const S3::Client & client_, const std::string & version_id_, const std::vector & keys_, const String & bucket_, @@ -134,7 +134,7 @@ public: UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, String compression_hint_, - const std::shared_ptr & client_, + const std::shared_ptr & client_, const String & bucket, const String & version_id, std::shared_ptr file_iterator_, @@ -155,7 +155,7 @@ private: UInt64 max_block_size; S3Settings::RequestSettings request_settings; String compression_hint; - std::shared_ptr client; + std::shared_ptr client; Block sample_block; std::optional format_settings; @@ -287,7 +287,7 @@ public: struct S3Configuration { const S3::URI uri; - std::shared_ptr client; + std::shared_ptr client; S3::AuthSettings auth_settings; S3Settings::RequestSettings request_settings; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 0ef02cac790..e158aff60a0 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -33,8 +33,6 @@ #include #include -#include -#include #include #include diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 79399589533..6e94e142bc2 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -100,6 +100,7 @@ SYSTEM DROP QUERY CACHE ['SYSTEM DROP QUERY','DROP QUERY CACHE','DROP QUERY'] GL SYSTEM DROP COMPILED EXPRESSION CACHE ['SYSTEM DROP COMPILED EXPRESSION','DROP COMPILED EXPRESSION CACHE','DROP COMPILED EXPRESSIONS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP FILESYSTEM CACHE ['SYSTEM DROP FILESYSTEM CACHE','DROP FILESYSTEM CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP SCHEMA CACHE ['SYSTEM DROP SCHEMA CACHE','DROP SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP S3 CLIENT CACHE ['SYSTEM DROP S3 CLIENT','DROP S3 CLIENT CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP CACHE ['DROP CACHE'] \N SYSTEM SYSTEM RELOAD CONFIG ['RELOAD CONFIG'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD USERS ['RELOAD USERS'] GLOBAL SYSTEM RELOAD diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4381e3d7dee..1aeb0bc75d0 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -288,7 +288,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP CACHE' = 102, 'SYSTEM RELOAD CONFIG' = 103, 'SYSTEM RELOAD USERS' = 104, 'SYSTEM RELOAD SYMBOLS' = 105, 'SYSTEM RELOAD DICTIONARY' = 106, 'SYSTEM RELOAD MODEL' = 107, 'SYSTEM RELOAD FUNCTION' = 108, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 109, 'SYSTEM RELOAD' = 110, 'SYSTEM RESTART DISK' = 111, 'SYSTEM MERGES' = 112, 'SYSTEM TTL MERGES' = 113, 'SYSTEM FETCHES' = 114, 'SYSTEM MOVES' = 115, 'SYSTEM DISTRIBUTED SENDS' = 116, 'SYSTEM REPLICATED SENDS' = 117, 'SYSTEM SENDS' = 118, 'SYSTEM REPLICATION QUEUES' = 119, 'SYSTEM DROP REPLICA' = 120, 'SYSTEM SYNC REPLICA' = 121, 'SYSTEM RESTART REPLICA' = 122, 'SYSTEM RESTORE REPLICA' = 123, 'SYSTEM WAIT LOADING PARTS' = 124, 'SYSTEM SYNC DATABASE REPLICA' = 125, 'SYSTEM SYNC TRANSACTION LOG' = 126, 'SYSTEM SYNC FILE CACHE' = 127, 'SYSTEM FLUSH DISTRIBUTED' = 128, 'SYSTEM FLUSH LOGS' = 129, 'SYSTEM FLUSH' = 130, 'SYSTEM THREAD FUZZER' = 131, 'SYSTEM UNFREEZE' = 132, 'SYSTEM' = 133, 'dictGet' = 134, 'addressToLine' = 135, 'addressToLineWithInlines' = 136, 'addressToSymbol' = 137, 'demangle' = 138, 'INTROSPECTION' = 139, 'FILE' = 140, 'URL' = 141, 'REMOTE' = 142, 'MONGO' = 143, 'MEILISEARCH' = 144, 'MYSQL' = 145, 'POSTGRES' = 146, 'SQLITE' = 147, 'ODBC' = 148, 'JDBC' = 149, 'HDFS' = 150, 'S3' = 151, 'HIVE' = 152, 'SOURCES' = 153, 'CLUSTER' = 154, 'ALL' = 155, 'NONE' = 156), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -569,10 +569,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP CACHE' = 102, 'SYSTEM RELOAD CONFIG' = 103, 'SYSTEM RELOAD USERS' = 104, 'SYSTEM RELOAD SYMBOLS' = 105, 'SYSTEM RELOAD DICTIONARY' = 106, 'SYSTEM RELOAD MODEL' = 107, 'SYSTEM RELOAD FUNCTION' = 108, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 109, 'SYSTEM RELOAD' = 110, 'SYSTEM RESTART DISK' = 111, 'SYSTEM MERGES' = 112, 'SYSTEM TTL MERGES' = 113, 'SYSTEM FETCHES' = 114, 'SYSTEM MOVES' = 115, 'SYSTEM DISTRIBUTED SENDS' = 116, 'SYSTEM REPLICATED SENDS' = 117, 'SYSTEM SENDS' = 118, 'SYSTEM REPLICATION QUEUES' = 119, 'SYSTEM DROP REPLICA' = 120, 'SYSTEM SYNC REPLICA' = 121, 'SYSTEM RESTART REPLICA' = 122, 'SYSTEM RESTORE REPLICA' = 123, 'SYSTEM WAIT LOADING PARTS' = 124, 'SYSTEM SYNC DATABASE REPLICA' = 125, 'SYSTEM SYNC TRANSACTION LOG' = 126, 'SYSTEM SYNC FILE CACHE' = 127, 'SYSTEM FLUSH DISTRIBUTED' = 128, 'SYSTEM FLUSH LOGS' = 129, 'SYSTEM FLUSH' = 130, 'SYSTEM THREAD FUZZER' = 131, 'SYSTEM UNFREEZE' = 132, 'SYSTEM' = 133, 'dictGet' = 134, 'addressToLine' = 135, 'addressToLineWithInlines' = 136, 'addressToSymbol' = 137, 'demangle' = 138, 'INTROSPECTION' = 139, 'FILE' = 140, 'URL' = 141, 'REMOTE' = 142, 'MONGO' = 143, 'MEILISEARCH' = 144, 'MYSQL' = 145, 'POSTGRES' = 146, 'SQLITE' = 147, 'ODBC' = 148, 'JDBC' = 149, 'HDFS' = 150, 'S3' = 151, 'HIVE' = 152, 'SOURCES' = 153, 'CLUSTER' = 154, 'ALL' = 155, 'NONE' = 156), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP CACHE' = 102, 'SYSTEM RELOAD CONFIG' = 103, 'SYSTEM RELOAD USERS' = 104, 'SYSTEM RELOAD SYMBOLS' = 105, 'SYSTEM RELOAD DICTIONARY' = 106, 'SYSTEM RELOAD MODEL' = 107, 'SYSTEM RELOAD FUNCTION' = 108, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 109, 'SYSTEM RELOAD' = 110, 'SYSTEM RESTART DISK' = 111, 'SYSTEM MERGES' = 112, 'SYSTEM TTL MERGES' = 113, 'SYSTEM FETCHES' = 114, 'SYSTEM MOVES' = 115, 'SYSTEM DISTRIBUTED SENDS' = 116, 'SYSTEM REPLICATED SENDS' = 117, 'SYSTEM SENDS' = 118, 'SYSTEM REPLICATION QUEUES' = 119, 'SYSTEM DROP REPLICA' = 120, 'SYSTEM SYNC REPLICA' = 121, 'SYSTEM RESTART REPLICA' = 122, 'SYSTEM RESTORE REPLICA' = 123, 'SYSTEM WAIT LOADING PARTS' = 124, 'SYSTEM SYNC DATABASE REPLICA' = 125, 'SYSTEM SYNC TRANSACTION LOG' = 126, 'SYSTEM SYNC FILE CACHE' = 127, 'SYSTEM FLUSH DISTRIBUTED' = 128, 'SYSTEM FLUSH LOGS' = 129, 'SYSTEM FLUSH' = 130, 'SYSTEM THREAD FUZZER' = 131, 'SYSTEM UNFREEZE' = 132, 'SYSTEM' = 133, 'dictGet' = 134, 'addressToLine' = 135, 'addressToLineWithInlines' = 136, 'addressToSymbol' = 137, 'demangle' = 138, 'INTROSPECTION' = 139, 'FILE' = 140, 'URL' = 141, 'REMOTE' = 142, 'MONGO' = 143, 'MEILISEARCH' = 144, 'MYSQL' = 145, 'POSTGRES' = 146, 'SQLITE' = 147, 'ODBC' = 148, 'JDBC' = 149, 'HDFS' = 150, 'S3' = 151, 'HIVE' = 152, 'SOURCES' = 153, 'CLUSTER' = 154, 'ALL' = 155, 'NONE' = 156)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 33877b5e006d48116854a9c0b64d7026013649f0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 3 Feb 2023 14:34:18 +0100 Subject: [PATCH 74/78] Parallel replicas. Part [2] (#43772) --- .github/workflows/pull_request.yml | 211 +++++++ docker/test/stateful/run.sh | 15 +- docker/test/stateless/run.sh | 6 +- programs/benchmark/Benchmark.cpp | 2 +- programs/copier/ClusterCopier.cpp | 2 +- programs/server/config.xml | 45 ++ src/CMakeLists.txt | 4 +- src/Client/Connection.cpp | 19 +- src/Client/Connection.h | 5 +- src/Client/HedgedConnections.h | 2 +- src/Client/IConnections.h | 4 +- src/Client/IServerConnection.h | 8 +- src/Client/LocalConnection.cpp | 2 +- src/Client/LocalConnection.h | 2 +- src/Client/MultiplexedConnections.cpp | 15 +- src/Client/MultiplexedConnections.h | 5 +- src/Core/Protocol.h | 4 +- src/Core/ProtocolDefines.h | 2 + src/Core/Settings.h | 2 + src/Disks/IDisk.h | 1 + .../ClusterProxy/SelectStreamFactory.cpp | 112 ++-- .../ClusterProxy/SelectStreamFactory.h | 18 +- .../ClusterProxy/executeQuery.cpp | 202 +++---- src/Interpreters/ClusterProxy/executeQuery.h | 5 +- src/Interpreters/Context.cpp | 44 ++ src/Interpreters/Context.h | 53 +- src/Interpreters/InterpreterSelectQuery.cpp | 30 +- src/Interpreters/InterpreterSelectQuery.h | 10 - .../InterpreterSelectQueryAnalyzer.cpp | 16 - src/Interpreters/interpretSubquery.cpp | 2 + .../QueryPlan/DistributedCreateLocalPlan.cpp | 53 +- .../QueryPlan/DistributedCreateLocalPlan.h | 8 +- .../Optimizations/optimizeReadInOrder.cpp | 2 - .../QueryPlan/Optimizations/optimizeTree.cpp | 5 +- src/Processors/QueryPlan/PartsSplitter.cpp | 3 +- src/Processors/QueryPlan/QueryPlan.cpp | 8 + .../QueryPlan/ReadFromMergeTree.cpp | 257 +++++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 15 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 65 +- src/Processors/QueryPlan/ReadFromRemote.h | 20 +- src/Processors/Sources/RemoteSource.cpp | 56 +- src/Processors/Sources/RemoteSource.h | 14 +- .../ReadFromMergeTreeDependencyTransform.cpp | 103 ++++ .../ReadFromMergeTreeDependencyTransform.h | 48 ++ src/QueryPipeline/Pipe.h | 2 + src/QueryPipeline/QueryPipelineBuilder.cpp | 102 +++- src/QueryPipeline/QueryPipelineBuilder.h | 6 + src/QueryPipeline/RemoteQueryExecutor.cpp | 81 ++- src/QueryPipeline/RemoteQueryExecutor.h | 61 +- src/Server/TCPHandler.cpp | 27 +- src/Server/TCPHandler.h | 6 +- src/Storages/MergeTree/IntersectionsIndexes.h | 2 +- src/Storages/MergeTree/MarkRange.cpp | 45 ++ src/Storages/MergeTree/MarkRange.h | 41 +- .../MergeTreeBaseSelectProcessor.cpp | 247 +------- .../MergeTree/MergeTreeBaseSelectProcessor.h | 47 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 6 +- .../MergeTree/MergeTreeBlockReadUtils.h | 15 +- src/Storages/MergeTree/MergeTreeData.cpp | 67 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +- .../MergeTreeInOrderSelectProcessor.cpp | 35 +- src/Storages/MergeTree/MergeTreePartInfo.cpp | 39 ++ src/Storages/MergeTree/MergeTreePartInfo.h | 6 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 270 +++++++-- src/Storages/MergeTree/MergeTreeReadPool.h | 200 +++++-- .../MergeTreeReverseSelectProcessor.cpp | 64 +- .../MergeTreeReverseSelectProcessor.h | 7 + .../MergeTree/MergeTreeSelectProcessor.cpp | 7 +- .../MergeTree/MergeTreeSelectProcessor.h | 8 +- .../MergeTreeThreadSelectProcessor.cpp | 79 +-- .../MergeTreeThreadSelectProcessor.h | 21 +- .../ParallelReplicasReadingCoordinator.cpp | 562 +++++++++++++++--- .../ParallelReplicasReadingCoordinator.h | 21 +- src/Storages/MergeTree/RangesInDataPart.cpp | 113 ++++ src/Storages/MergeTree/RangesInDataPart.h | 71 ++- src/Storages/MergeTree/RequestResponse.cpp | 188 +++--- src/Storages/MergeTree/RequestResponse.h | 52 +- .../MergeTree/tests/gtest_coordinator.cpp | 240 -------- src/Storages/SelectQueryInfo.h | 17 +- src/Storages/StorageDistributed.cpp | 78 +-- src/Storages/StorageMergeTree.cpp | 41 +- src/Storages/StorageProxy.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 51 +- src/Storages/getStructureOfRemoteTable.cpp | 4 +- tests/ci/ci_config.py | 20 + tests/ci/functional_test_check.py | 45 +- tests/clickhouse-test | 11 + tests/performance/memory_bound_merging.xml | 2 - .../02404_memory_bound_merging.reference | 38 +- .../02404_memory_bound_merging.sql | 31 +- .../1_stateful/00009_uniq_distributed.sql | 1 + .../1_stateful/00012_sorting_distributed.sql | 1 + .../1_stateful/00013_sorting_of_nested.sql | 2 + ...6_any_if_distributed_cond_always_false.sql | 1 + .../1_stateful/00022_merge_prewhere.sql | 2 + .../1_stateful/00042_any_left_join.sql | 2 + .../1_stateful/00043_any_left_join.sql | 2 + .../1_stateful/00044_any_left_join_string.sql | 2 + .../1_stateful/00063_loyalty_joins.sql | 46 +- .../00065_loyalty_with_storage_join.sql | 4 +- ...0066_sorting_distributed_many_replicas.sql | 1 + tests/queries/1_stateful/00074_full_join.sql | 2 + .../1_stateful/00075_left_array_join.sql | 2 + ...0079_array_join_not_used_joined_column.sql | 2 + .../1_stateful/00080_array_join_and_union.sql | 2 + .../1_stateful/00084_external_aggregation.sql | 2 +- .../00091_prewhere_two_conditions.sql | 3 + tests/queries/1_stateful/00092_obfuscator.sh | 2 + .../1_stateful/00096_obfuscator_save_load.sh | 2 + .../00146_aggregate_function_uniq.sql | 2 + .../00149_quantiles_timing_distributed.sql | 2 +- .../00152_insert_different_granularity.sql | 2 +- ...00156_max_execution_speed_sample_merge.sql | 3 + .../1_stateful/00166_explain_estimate.sql | 2 +- tests/queries/1_stateful/00170_s3_cache.sql | 2 +- ...0171_grouping_aggregated_transform_bug.sql | 2 +- .../00172_early_constant_folding.sql | 2 + .../1_stateful/00172_hits_joins.sql.j2 | 1 + .../00175_obfuscator_schema_inference.sh | 1 + 119 files changed, 3019 insertions(+), 1668 deletions(-) create mode 100644 src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.cpp create mode 100644 src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.h create mode 100644 src/Storages/MergeTree/RangesInDataPart.cpp delete mode 100644 src/Storages/MergeTree/tests/gtest_coordinator.cpp diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c677ec4bf5c..40424c7a08c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2813,6 +2813,217 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + # Parallel replicas + FunctionalStatefulTestDebugParallelReplicas: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestUBsanParallelReplicas: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestMsanParallelReplicas: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (msan, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestTsanParallelReplicas: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (tsan, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAsanParallelReplicas: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (asan, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestReleaseParallelReplicas: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, ParallelReplicas) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################## ######################################### STRESS TESTS ####################################### ############################################################################################## diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index e69a85c0fca..80a43799914 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -126,13 +126,16 @@ function run_tests() fi set +e - clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \ - --skip 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" \ + + if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then + clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 \ + --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \ + -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt - - clickhouse-test --timeout 1200 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \ - 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a test_output/test_result.txt - + else + clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + fi set -e } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index fef3fc4d228..8e7d0ef55b9 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -134,9 +134,9 @@ function run_tests() set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ - | ts '%Y-%m-%d %H:%M:%S' \ - | tee -a test_output/test_result.txt + --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + | ts '%Y-%m-%d %H:%M:%S' \ + | tee -a test_output/test_result.txt set -e } diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index dae3aea2d2e..7fb0b1f154f 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -474,7 +474,7 @@ private: executor.sendQuery(ClientInfo::QueryKind::INITIAL_QUERY); ProfileInfo info; - while (Block block = executor.read()) + while (Block block = executor.readBlock()) info.update(block); executor.finish(); diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 256b40414c5..2fc0eb27213 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -2040,7 +2040,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( while (true) { - auto block = remote_query_executor->read(); + auto block = remote_query_executor->readBlock(); if (!block) break; } diff --git a/programs/server/config.xml b/programs/server/config.xml index bd46263f851..b860260834f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -854,6 +854,51 @@ + + + false + + 127.0.0.1 + 9000 + + + 127.0.0.2 + 9000 + + + 127.0.0.3 + 9000 + + + 127.0.0.4 + 9000 + + + 127.0.0.5 + 9000 + + + 127.0.0.6 + 9000 + + + 127.0.0.7 + 9000 + + + 127.0.0.8 + 9000 + + + 127.0.0.9 + 9000 + + + 127.0.0.10 + 9000 + + + diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 14838560a88..a74f10d2c3c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -343,8 +343,8 @@ set_source_files_properties( PROPERTIES COMPILE_FLAGS "-mwaitpkg") endif () -target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2_st) -target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2) +target_link_libraries(common PUBLIC ch_contrib::re2_st) +target_link_libraries(common PUBLIC ch_contrib::re2) target_link_libraries(clickhouse_common_io PUBLIC diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 701e26e4f67..eea007a8608 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -686,7 +686,7 @@ void Connection::sendReadTaskResponse(const String & response) } -void Connection::sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) +void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); response.serialize(*out); @@ -960,8 +960,12 @@ Packet Connection::receivePacket() case Protocol::Server::ReadTaskRequest: return res; + case Protocol::Server::MergeTreeAllRangesAnnounecement: + res.announcement = receiveInitialParallelReadAnnounecement(); + return res; + case Protocol::Server::MergeTreeReadTaskRequest: - res.request = receivePartitionReadRequest(); + res.request = receiveParallelReadRequest(); return res; case Protocol::Server::ProfileEvents: @@ -1114,13 +1118,20 @@ ProfileInfo Connection::receiveProfileInfo() const return profile_info; } -PartitionReadRequest Connection::receivePartitionReadRequest() const +ParallelReadRequest Connection::receiveParallelReadRequest() const { - PartitionReadRequest request; + ParallelReadRequest request; request.deserialize(*in); return request; } +InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnounecement() const +{ + InitialAllRangesAnnouncement announcement; + announcement.deserialize(*in); + return announcement; +} + void Connection::throwUnexpectedPacket(UInt64 packet_type, const char * expected) const { diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 5128757f11c..d806c5e8b1f 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -110,7 +110,7 @@ public: void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override; - void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override; + void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; void sendExternalTablesData(ExternalTablesData & data) override; @@ -265,7 +265,8 @@ private: std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; Progress receiveProgress() const; - PartitionReadRequest receivePartitionReadRequest() const; + ParallelReadRequest receiveParallelReadRequest() const; + InitialAllRangesAnnouncement receiveInitialParallelReadAnnounecement() const; ProfileInfo receiveProfileInfo() const; void initInputBuffers(); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index fe697c78bb7..40f031a16a6 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -94,7 +94,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "sendReadTaskResponse in not supported with HedgedConnections"); } - void sendMergeTreeReadTaskResponse(PartitionReadResponse) override + void sendMergeTreeReadTaskResponse(const ParallelReadResponse &) override { throw Exception(ErrorCodes::LOGICAL_ERROR, "sendMergeTreeReadTaskResponse in not supported with HedgedConnections"); } diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index 9d695e6da92..0040eeb31ed 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -34,7 +34,7 @@ public: bool with_pending_data) = 0; virtual void sendReadTaskResponse(const String &) = 0; - virtual void sendMergeTreeReadTaskResponse(PartitionReadResponse response) = 0; + virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0; /// Get packet from any replica. virtual Packet receivePacket() = 0; @@ -60,9 +60,9 @@ public: /// Get the replica addresses as a string. virtual std::string dumpAddresses() const = 0; - struct ReplicaInfo { + bool collaborate_with_initiator{false}; size_t all_replicas_count{0}; size_t number_of_current_replica{0}; }; diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index d75ea3a713c..cd4db8f5258 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -33,8 +33,10 @@ struct Packet Progress progress; ProfileInfo profile_info; std::vector part_uuids; - PartitionReadRequest request; - PartitionReadResponse response; + + InitialAllRangesAnnouncement announcement; + ParallelReadRequest request; + ParallelReadResponse response; Packet() : type(Protocol::Server::Hello) {} }; @@ -104,7 +106,7 @@ public: /// Send all contents of external (temporary) tables. virtual void sendExternalTablesData(ExternalTablesData & data) = 0; - virtual void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) = 0; + virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0; /// Check, if has data to read. virtual bool poll(size_t timeout_microseconds) = 0; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index d86a097b910..77a0a846300 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -508,7 +508,7 @@ void LocalConnection::sendExternalTablesData(ExternalTablesData &) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } -void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse &) +void LocalConnection::sendMergeTreeReadTaskResponse(const ParallelReadResponse &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index cfdffefc0ce..3e6fc007fb9 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -110,7 +110,7 @@ public: void sendExternalTablesData(ExternalTablesData &) override; - void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override; + void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; bool poll(size_t timeout_microseconds/* = 0 */) override; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index a10846657d6..cc260353339 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -133,16 +133,11 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - bool parallel_reading_from_replicas = settings.max_parallel_replicas > 1 - && settings.allow_experimental_parallel_reading_from_replicas - /// To avoid trying to coordinate with clickhouse-benchmark, - /// since it uses the same code. - && client_info.query_kind != ClientInfo::QueryKind::INITIAL_QUERY; - if (parallel_reading_from_replicas) + if (replica_info) { client_info.collaborate_with_initiator = true; - client_info.count_participating_replicas = replica_info.all_replicas_count; - client_info.number_of_current_replica = replica_info.number_of_current_replica; + client_info.count_participating_replicas = replica_info->all_replicas_count; + client_info.number_of_current_replica = replica_info->number_of_current_replica; } } @@ -199,7 +194,7 @@ void MultiplexedConnections::sendReadTaskResponse(const String & response) } -void MultiplexedConnections::sendMergeTreeReadTaskResponse(PartitionReadResponse response) +void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { std::lock_guard lock(cancel_mutex); if (cancelled) @@ -263,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: case Protocol::Server::PartUUIDs: @@ -343,6 +339,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: case Protocol::Server::PartUUIDs: diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index e76d54218c7..dd228067ed2 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -42,7 +42,7 @@ public: bool with_pending_data) override; void sendReadTaskResponse(const String &) override; - void sendMergeTreeReadTaskResponse(PartitionReadResponse response) override; + void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; Packet receivePacket() override; @@ -104,7 +104,8 @@ private: bool sent_query = false; bool cancelled = false; - ReplicaInfo replica_info; + /// std::nullopt if parallel reading from replicas is not used + std::optional replica_info; /// A mutex for the sendCancel function to execute safely /// in separate thread. diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 08c675eb421..86c0a851c60 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -81,7 +81,8 @@ namespace Protocol /// This is such an inverted logic, where server sends requests /// And client returns back response ProfileEvents = 14, /// Packet with profile events from server. - MergeTreeReadTaskRequest = 15, /// Request from a MergeTree replica to a coordinator + MergeTreeAllRangesAnnounecement = 15, + MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator MAX = MergeTreeReadTaskRequest, }; @@ -108,6 +109,7 @@ namespace Protocol "PartUUIDs", "ReadTaskRequest", "ProfileEvents", + "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", }; return packet <= MAX diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 01017a2b751..3bbfb95f020 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -33,6 +33,8 @@ #define DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION 1 #define DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS 54453 +#define DBMS_MERGE_TREE_PART_INFO_VERSION 1 + /// Minimum revision supporting interserver secret. #define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 049552a2bb8..bbce7bda0dd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -151,7 +151,9 @@ class IColumn; M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ \ + M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 7907c0b2a74..460ce14f197 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -28,6 +28,7 @@ namespace Poco { namespace Util { + /// NOLINTNEXTLINE(cppcoreguidelines-virtual-class-destructor) class AbstractConfiguration; } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 5c781c531ed..b08ec7e5ab5 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -7,7 +7,10 @@ #include #include #include +#include +#include #include +#include #include #include @@ -36,6 +39,53 @@ namespace ErrorCodes namespace ClusterProxy { +/// select query has database, table and table function names as AST pointers +/// Creates a copy of query, changes database, table and table function names. +ASTPtr rewriteSelectQuery( + ContextPtr context, + const ASTPtr & query, + const std::string & remote_database, + const std::string & remote_table, + ASTPtr table_function_ptr) +{ + auto modified_query_ast = query->clone(); + + ASTSelectQuery & select_query = modified_query_ast->as(); + + // Get rid of the settings clause so we don't send them to remote. Thus newly non-important + // settings won't break any remote parser. It's also more reasonable since the query settings + // are written into the query context and will be sent by the query pipeline. + select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {}); + + if (table_function_ptr) + select_query.addTableFunction(table_function_ptr); + else + select_query.replaceDatabaseAndTable(remote_database, remote_table); + + /// Restore long column names (cause our short names are ambiguous). + /// TODO: aliased table functions & CREATE TABLE AS table function cases + if (!table_function_ptr) + { + RestoreQualifiedNamesVisitor::Data data; + data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as(), 0)); + data.remote_table.database = remote_database; + data.remote_table.table = remote_table; + RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); + } + + /// To make local JOIN works, default database should be added to table names. + /// But only for JOIN section, since the following should work using default_database: + /// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value + /// (see 01487_distributed_in_not_default_db) + AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(), + /* only_replace_current_database_function_= */false, + /* only_replace_in_join_= */true); + visitor.visit(modified_query_ast); + + return modified_query_ast; +} + + SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, @@ -171,67 +221,5 @@ void SelectStreamFactory::createForShard( } -void SelectStreamFactory::createForShardWithParallelReplicas( - const Cluster::ShardInfo & shard_info, - const ASTPtr & query_ast, - const StorageID & main_table, - ContextPtr context, - UInt32 shard_count, - std::vector & local_plans, - Shards & remote_shards) -{ - if (auto it = objects_by_shard.find(shard_info.shard_num); it != objects_by_shard.end()) - replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast); - - const auto & settings = context->getSettingsRef(); - - auto is_local_replica_obsolete = [&]() - { - auto resolved_id = context->resolveStorageID(main_table); - auto main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context); - const auto * replicated_storage = dynamic_cast(main_table_storage.get()); - - if (!replicated_storage) - return false; - - UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; - - if (!max_allowed_delay) - return false; - - UInt64 local_delay = replicated_storage->getAbsoluteDelay(); - return local_delay >= max_allowed_delay; - }; - - size_t next_replica_number = 0; - size_t all_replicas_count = shard_info.getRemoteNodeCount(); - - auto coordinator = std::make_shared(); - - if (settings.prefer_localhost_replica && shard_info.isLocal()) - { - /// We don't need more than one local replica in parallel reading - if (!is_local_replica_obsolete()) - { - ++all_replicas_count; - - local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, next_replica_number, all_replicas_count, coordinator)); - - ++next_replica_number; - } - } - - if (shard_info.hasRemoteConnections()) - remote_shards.emplace_back(Shard{ - .query = query_ast, - .header = header, - .shard_info = shard_info, - .lazy = false, - .local_delay = 0, - .coordinator = coordinator, - }); -} - } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index a8f7d131b15..f1a8b3e0984 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -29,6 +29,14 @@ struct StorageID; namespace ClusterProxy { +/// select query has database, table and table function names as AST pointers +/// Creates a copy of query, changes database, table and table function names. +ASTPtr rewriteSelectQuery( + ContextPtr context, + const ASTPtr & query, + const std::string & remote_database, + const std::string & remote_table, + ASTPtr table_function_ptr = nullptr); using ColumnsDescriptionByShardNum = std::unordered_map; @@ -80,16 +88,6 @@ public: std::unique_ptr remote_plan; }; - void createForShardWithParallelReplicas( - const Cluster::ShardInfo & shard_info, - const ASTPtr & query_ast, - const StorageID & main_table, - ContextPtr context, - UInt32 shard_count, - std::vector & local_plans, - Shards & remote_shards); - -private: const Block header; const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index fe31b4d8302..2e035ef883f 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -1,6 +1,8 @@ #include #include +#include #include +#include #include #include #include @@ -13,8 +15,11 @@ #include #include #include +#include +#include #include #include +#include namespace DB { @@ -23,6 +28,7 @@ namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; + extern const int SUPPORT_IS_DISABLED; } namespace ClusterProxy @@ -117,6 +123,31 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c return new_context; } +static ThrottlerPtr getThrottler(const ContextPtr & context) +{ + const Settings & settings = context->getSettingsRef(); + + ThrottlerPtr user_level_throttler; + if (auto process_list_element = context->getProcessListElement()) + user_level_throttler = process_list_element->getUserNetworkThrottler(); + + /// Network bandwidth limit, if needed. + ThrottlerPtr throttler; + if (settings.max_network_bandwidth || settings.max_network_bytes) + { + throttler = std::make_shared( + settings.max_network_bandwidth, + settings.max_network_bytes, + "Limit for bytes to send or receive over network exceeded.", + user_level_throttler); + } + else + throttler = user_level_throttler; + + return throttler; +} + + void executeQuery( QueryPlan & query_plan, const Block & header, @@ -138,26 +169,8 @@ void executeQuery( SelectStreamFactory::Shards remote_shards; auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log); - new_context->getClientInfo().distributed_depth += 1; - ThrottlerPtr user_level_throttler; - if (auto process_list_element = context->getProcessListElement()) - user_level_throttler = process_list_element->getUserNetworkThrottler(); - - /// Network bandwidth limit, if needed. - ThrottlerPtr throttler; - if (settings.max_network_bandwidth || settings.max_network_bytes) - { - throttler = std::make_shared( - settings.max_network_bandwidth, - settings.max_network_bytes, - "Limit for bytes to send or receive over network exceeded.", - user_level_throttler); - } - else - throttler = user_level_throttler; - size_t shards = query_info.getCluster()->getShardCount(); for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) { @@ -199,7 +212,7 @@ void executeQuery( main_table, table_func_ptr, new_context, - throttler, + getThrottler(context), std::move(scalars), std::move(external_tables), log, @@ -236,103 +249,76 @@ void executeQueryWithParallelReplicas( const StorageID & main_table, const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, - const ASTPtr & query_ast, - ContextPtr context, - const SelectQueryInfo & query_info, - const ExpressionActionsPtr & sharding_key_expr, - const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster, - QueryProcessingStage::Enum processed_stage) + const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, + const ClusterPtr & not_optimized_cluster) { - const Settings & settings = context->getSettingsRef(); + if (not_optimized_cluster->getShardsInfo().size() != 1) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Cluster for parallel replicas should consist only from one shard"); - ThrottlerPtr user_level_throttler; - if (auto process_list_element = context->getProcessListElement()) - user_level_throttler = process_list_element->getUserNetworkThrottler(); + auto shard_info = not_optimized_cluster->getShardsInfo().front(); - /// Network bandwidth limit, if needed. - ThrottlerPtr throttler; - if (settings.max_network_bandwidth || settings.max_network_bytes) - { - throttler = std::make_shared( - settings.max_network_bandwidth, - settings.max_network_bytes, - "Limit for bytes to send or receive over network exceeded.", - user_level_throttler); - } - else - throttler = user_level_throttler; - - - std::vector plans; - SelectStreamFactory::Shards remote_shards; - size_t shards = query_info.getCluster()->getShardCount(); - - for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) - { - ASTPtr query_ast_for_shard; - if (query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) - { - query_ast_for_shard = query_ast->clone(); - - OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ - sharding_key_expr, - sharding_key_expr->getSampleBlock().getByPosition(0).type, - sharding_key_column_name, - shard_info, - not_optimized_cluster->getSlotToShard(), - }; - OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); - visitor.visit(query_ast_for_shard); - } - else - query_ast_for_shard = query_ast; - - stream_factory.createForShardWithParallelReplicas( - shard_info, query_ast_for_shard, main_table, context, static_cast(shards), plans, remote_shards); - } - - Scalars scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; - scalars.emplace( - "_shard_count", Block{{DataTypeUInt32().createColumnConst(1, shards), std::make_shared(), "_shard_count"}}); - auto external_tables = context->getExternalTables(); - - if (!remote_shards.empty()) - { - auto new_context = Context::createCopy(context); - - for (const auto & shard : remote_shards) - { - auto read_from_remote = std::make_unique( - shard.coordinator, - shard, - shard.header, - processed_stage, - main_table, - table_func_ptr, - new_context, - throttler, - scalars, - external_tables, - &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), - query_info.storage_limits); - - auto remote_plan = std::make_unique(); - remote_plan->addStep(std::move(read_from_remote)); - remote_plan->addInterpreterContext(new_context); - plans.emplace_back(std::move(remote_plan)); - } - } - - if (plans.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from Distributed. This is a bug"); - - if (plans.size() == 1) + const auto & settings = context->getSettingsRef(); + auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), shard_info.all_addresses.size()); + auto coordinator = std::make_shared(all_replicas_count); + auto remote_plan = std::make_unique(); + auto plans = std::vector(); + + /// This is a little bit weird, but we construct an "empty" coordinator without + /// any specified reading/coordination method (like Default, InOrder, InReverseOrder) + /// Because we will understand it later during QueryPlan optimization + /// So we place a reference to the coordinator to some common plane like QueryInfo + /// to then tell it about the reading method we chose. + query_info.coordinator = coordinator; + + UUID parallel_group_id = UUIDHelpers::generateV4(); + + plans.emplace_back(createLocalPlan( + query_ast, + stream_factory.header, + context, + stream_factory.processed_stage, + shard_info.shard_num, + /*shard_count*/1, + 0, + all_replicas_count, + coordinator, + parallel_group_id)); + + if (!shard_info.hasRemoteConnections()) { + if (!plans.front()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "An empty plan was generated to read from local shard and there is no remote connections. This is a bug"); query_plan = std::move(*plans.front()); return; } + auto new_context = Context::createCopy(context); + auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; + auto external_tables = new_context->getExternalTables(); + + auto read_from_remote = std::make_unique( + query_ast, + std::move(shard_info), + coordinator, + stream_factory.header, + stream_factory.processed_stage, + main_table, + table_func_ptr, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), + query_info.storage_limits, + parallel_group_id); + + remote_plan->addStep(std::move(read_from_remote)); + remote_plan->addInterpreterContext(context); + plans.emplace_back(std::move(remote_plan)); + + if (std::all_of(plans.begin(), plans.end(), [](const QueryPlanPtr & plan) { return !plan; })) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug"); + DataStreams input_streams; input_streams.reserve(plans.size()); for (const auto & plan : plans) diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 662fe47ca65..787e79313cc 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -61,10 +61,7 @@ void executeQueryWithParallelReplicas( const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, - const ExpressionActionsPtr & sharding_key_expr, - const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster, - QueryProcessingStage::Enum processed_stage); + const ClusterPtr & not_optimized_cluster); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d9a7aa2e677..4032590de71 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3621,6 +3621,32 @@ void Context::setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback merge_tree_read_task_callback = callback; } + +MergeTreeAllRangesCallback Context::getMergeTreeAllRangesCallback() const +{ + if (!merge_tree_all_ranges_callback.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Next task callback is not set for query with id: {}", getInitialQueryId()); + + return merge_tree_all_ranges_callback.value(); +} + + +void Context::setMergeTreeAllRangesCallback(MergeTreeAllRangesCallback && callback) +{ + merge_tree_all_ranges_callback = callback; +} + + +void Context::setParallelReplicasGroupUUID(UUID uuid) +{ + parallel_replicas_group_uuid = uuid; +} + +UUID Context::getParallelReplicasGroupUUID() const +{ + return parallel_replicas_group_uuid; +} + PartUUIDsPtr Context::getIgnoredPartUUIDs() const { auto lock = getLock(); @@ -3886,4 +3912,22 @@ WriteSettings Context::getWriteSettings() const return res; } +bool Context::canUseParallelReplicasOnInitiator() const +{ + const auto & settings = getSettingsRef(); + return settings.allow_experimental_parallel_reading_from_replicas + && settings.max_parallel_replicas > 1 + && !settings.use_hedged_requests + && !getClientInfo().collaborate_with_initiator; +} + +bool Context::canUseParallelReplicasOnFollower() const +{ + const auto & settings = getSettingsRef(); + return settings.allow_experimental_parallel_reading_from_replicas + && settings.max_parallel_replicas > 1 + && !settings.use_hedged_requests + && getClientInfo().collaborate_with_initiator; +} + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 00dc4204496..d0eab42fec2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,11 @@ #pragma once +#include +#include +#include +#include +#include +#include #include #include #include @@ -8,32 +14,24 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include - +#include +#include +#include +#include #include - +#include +#include #include "config.h" #include +#include #include #include #include #include - #include -#include namespace Poco::Net { class IPAddress; } @@ -98,7 +96,11 @@ class TransactionsInfoLog; class ProcessorsProfileLog; class FilesystemCacheLog; class AsynchronousInsertLog; +class IAsynchronousReader; struct MergeTreeSettings; +struct InitialAllRangesAnnouncement; +struct ParallelReadRequest; +struct ParallelReadResponse; class StorageS3Settings; class IDatabase; class DDLWorker; @@ -172,11 +174,15 @@ using InputBlocksReader = std::function; /// Used in distributed task processing using ReadTaskCallback = std::function; -using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; +using MergeTreeAllRangesCallback = std::function; +using MergeTreeReadTaskCallback = std::function(ParallelReadRequest)>; class TemporaryDataOnDiskScope; using TemporaryDataOnDiskScopePtr = std::shared_ptr; +class ParallelReplicasReadingCoordinator; +using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr; + #if USE_ROCKSDB class MergeTreeMetadataCache; using MergeTreeMetadataCachePtr = std::shared_ptr; @@ -262,6 +268,8 @@ private: /// Used in parallel reading from replicas. A replica tells about its intentions to read /// some ranges from some part and initiator will tell the replica about whether it is accepted or denied. std::optional merge_tree_read_task_callback; + std::optional merge_tree_all_ranges_callback; + UUID parallel_replicas_group_uuid{UUIDHelpers::Nil}; /// Record entities accessed by current query, and store this information in system.query_log. struct QueryAccessInfo @@ -380,6 +388,7 @@ private: /// Temporary data for query execution accounting. TemporaryDataOnDiskScopePtr temp_data_on_disk; + public: /// Some counters for current query execution. /// Most of them are workarounds and should be removed in the future. @@ -402,6 +411,8 @@ public: KitchenSink kitchen_sink; + ParallelReplicasReadingCoordinatorPtr parallel_reading_coordinator; + private: using SampleBlockCache = std::unordered_map; mutable SampleBlockCache sample_block_cache; @@ -1045,6 +1056,12 @@ public: MergeTreeReadTaskCallback getMergeTreeReadTaskCallback() const; void setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback); + MergeTreeAllRangesCallback getMergeTreeAllRangesCallback() const; + void setMergeTreeAllRangesCallback(MergeTreeAllRangesCallback && callback); + + UUID getParallelReplicasGroupUUID() const; + void setParallelReplicasGroupUUID(UUID uuid); + /// Background executors related methods void initializeBackgroundExecutorsIfNeeded(); bool areBackgroundExecutorsInitialized(); @@ -1071,6 +1088,10 @@ public: /** Get settings for writing to filesystem. */ WriteSettings getWriteSettings() const; + /** There are multiple conditions that have to be met to be able to use parallel replicas */ + bool canUseParallelReplicasOnInitiator() const; + bool canUseParallelReplicasOnFollower() const; + private: std::unique_lock getLock() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e11da479c0..2ce116f7796 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -448,6 +448,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// FIXME: Memory bound aggregation may cause another reading algorithm to be used on remote replicas + if (settings.allow_experimental_parallel_reading_from_replicas && settings.enable_memory_bound_merging_of_aggregation_results) + context->setSetting("enable_memory_bound_merging_of_aggregation_results", false); + + if (joined_tables.tablesCount() > 1 && settings.allow_experimental_parallel_reading_from_replicas) + { + LOG_WARNING(log, "Joins are not supported with parallel replicas. Query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", false); + } + /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { @@ -543,6 +553,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( parameter_values, parameter_types); + query_info.syntax_analyzer_result = syntax_analyzer_result; context->setDistributed(syntax_analyzer_result->is_remote_storage); @@ -1902,22 +1913,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( } } -void InterpreterSelectQuery::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback) -{ - context->getClientInfo().collaborate_with_initiator = true; - context->setMergeTreeReadTaskCallback(std::move(callback)); -} - -void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t replica_count) -{ - context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context->getClientInfo().count_participating_replicas = replica_count; - context->getClientInfo().number_of_current_replica = replica_num; - context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR; - context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR; - context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; -} - RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const { return row_policy_filter; @@ -2572,12 +2567,13 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool const bool should_produce_results_in_order_of_bucket_number = options.to_stage == QueryProcessingStage::WithMergeableState && (settings.distributed_aggregation_memory_efficient || settings.enable_memory_bound_merging_of_aggregation_results); + const bool parallel_replicas_from_merge_tree = storage->isMergeTree() && context->canUseParallelReplicasOnInitiator(); executeMergeAggregatedImpl( query_plan, overflow_row, final, - storage && storage->isRemote(), + storage && (storage->isRemote() || parallel_replicas_from_merge_tree), has_grouping_sets, context->getSettingsRef(), query_analyzer->aggregationKeys(), diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 761eea8e1b8..0ab1ba58e0f 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -122,16 +122,6 @@ public: bool supportsTransactions() const override { return true; } - /// This is tiny crutch to support reading from localhost replica during distributed query - /// Replica need to talk to the initiator through a connection to ask for a next task - /// but there will be no connection if we create Interpreter explicitly. - /// The other problem is that context is copied inside Interpreter's constructor - /// And with this method we can change the internals of cloned one - void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback); - - /// It will set shard_num and shard_count to the client_info - void setProperClientInfo(size_t replica_num, size_t replica_count); - FilterDAGInfoPtr getAdditionalQueryInfo() const { return additional_filter_info; } RowPolicyFilterPtr getRowPolicyFilter() const; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 85bc70e9382..6b6f3560c5a 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -135,20 +135,4 @@ void InterpreterSelectQueryAnalyzer::addStorageLimits(const StorageLimitsList & planner.addStorageLimits(storage_limits); } -void InterpreterSelectQueryAnalyzer::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback) -{ - context->getClientInfo().collaborate_with_initiator = true; - context->setMergeTreeReadTaskCallback(std::move(callback)); -} - -void InterpreterSelectQueryAnalyzer::setProperClientInfo(size_t replica_number, size_t count_participating_replicas) -{ - context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - context->getClientInfo().number_of_current_replica = replica_number; - context->getClientInfo().count_participating_replicas = count_participating_replicas; - context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR; - context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR; - context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; -} - } diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 5f00be07fa5..2358b0ab42a 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -112,6 +112,8 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } + /// We don't want to execute reading for subqueries in parallel + subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", false); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 166b021b5ce..2bb29a0b6fe 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -1,9 +1,12 @@ #include + +#include "config_version.h" #include -#include +#include #include #include #include +#include namespace DB { @@ -40,48 +43,58 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - UInt32 shard_num, - UInt32 shard_count, + size_t shard_num, + size_t shard_count, size_t replica_num, size_t replica_count, - std::shared_ptr coordinator) + std::shared_ptr coordinator, + UUID group_uuid) { checkStackSize(); auto query_plan = std::make_unique(); + auto new_context = Context::createCopy(context); /// Do not apply AST optimizations, because query /// is already optimized and some optimizations /// can be applied only for non-distributed tables /// and we can produce query, inconsistent with remote plans. auto select_query_options = SelectQueryOptions(processed_stage) - .setShardInfo(shard_num, shard_count) + .setShardInfo(static_cast(shard_num), static_cast(shard_count)) .ignoreASTOptimizations(); - auto update_interpreter = [&](auto & interpreter) + /// There are much things that are needed for coordination + /// during reading with parallel replicas + if (coordinator) { - interpreter.setProperClientInfo(replica_num, replica_count); - if (coordinator) + new_context->parallel_reading_coordinator = coordinator; + new_context->getClientInfo().interface = ClientInfo::Interface::LOCAL; + new_context->getClientInfo().collaborate_with_initiator = true; + new_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + new_context->getClientInfo().count_participating_replicas = replica_count; + new_context->getClientInfo().number_of_current_replica = replica_num; + new_context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR; + new_context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR; + new_context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; + new_context->setParallelReplicasGroupUUID(group_uuid); + new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement) { - interpreter.setMergeTreeReadTaskCallbackAndClientInfo([coordinator](PartitionReadRequest request) -> std::optional - { - return coordinator->handleRequest(request); - }); - } - }; + coordinator->handleInitialAllRangesAnnouncement(announcement); + }); + new_context->setMergeTreeReadTaskCallback([coordinator](ParallelReadRequest request) -> std::optional + { + return coordinator->handleRequest(request); + }); + } if (context->getSettingsRef().allow_experimental_analyzer) { - auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, context, select_query_options); - update_interpreter(interpreter); + auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); } else { - auto interpreter = InterpreterSelectQuery( - query_ast, context, - select_query_options); - update_interpreter(interpreter); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index b55cedf9871..16bf1c565ff 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -13,10 +14,11 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - UInt32 shard_num, - UInt32 shard_count, + size_t shard_num, + size_t shard_count, size_t replica_num, size_t replica_count, - std::shared_ptr coordinator); + std::shared_ptr coordinator, + UUID group_uuid = UUIDHelpers::Nil); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index bdf8f24f9d6..301c3bca571 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -1005,8 +1005,6 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n if (auto * reading = typeid_cast(reading_node->step.get())) { - - //std::cerr << "---- optimizeReadInOrder found mt" << std::endl; auto order_info = buildInputOrderInfo( reading, fixed_columns, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 13095dfad47..0378c5ef416 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -1,8 +1,9 @@ -#include -#include #include #include +#include +#include #include + #include namespace DB diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index e7ea7a4e34b..917bea4c884 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -13,6 +13,7 @@ #include #include #include +#include using namespace DB; @@ -77,7 +78,7 @@ std::pair, std::vector> split(RangesInDat RangeEnd, }; - bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); } + [[ maybe_unused ]] bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); } Values value; MarkRangeWithPartIdx range; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index e817a9ef8a9..48a9fbd7a34 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -166,6 +166,7 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( QueryPipelineBuilderPtr last_pipeline; + std::stack stack; stack.push(Frame{.node = root}); @@ -198,6 +199,13 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( last_pipeline->setProcessListElement(build_pipeline_settings.process_list_element); last_pipeline->addResources(std::move(resources)); + /// This is related to parallel replicas. + /// Not to let the remote sources starve for CPU we create an + /// explicit dependency between processors which read from local replica + /// and ones that receive data from remote replicas and constantly answer + /// to coordination packets. + last_pipeline->connectDependencies(); + return last_pipeline; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 22245b82966..cca8e5297ee 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,16 +1,18 @@ -#include -#include -#include -#include -#include -#include -#include +#include + +#include +#include +#include +#include +#include "Storages/MergeTree/RequestResponse.h" +#include #include #include -#include +#include #include #include #include +#include #include #include #include @@ -25,18 +27,22 @@ #include #include #include +#include #include #include #include -#include #include #include #include +#include #include -#include -#include -#include -#include + +#include +#include +#include +#include +#include +#include namespace ProfileEvents { @@ -114,6 +120,7 @@ ReadFromMergeTree::ReadFromMergeTree( , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(log_) , analyzed_result_ptr(analyzed_result_ptr_) + , is_parallel_reading_from_replicas(enable_parallel_reading) { if (sample_factor_column_queried) { @@ -123,8 +130,11 @@ ReadFromMergeTree::ReadFromMergeTree( output_stream->header.insert({type->createColumn(), type, "_sample_factor"}); } - if (enable_parallel_reading) + if (is_parallel_reading_from_replicas) + { + all_ranges_callback = context->getMergeTreeAllRangesCallback(); read_task_callback = context->getMergeTreeReadTaskCallback(); + } const auto & settings = context->getSettingsRef(); if (settings.max_streams_for_merge_tree_reading) @@ -173,6 +183,80 @@ ReadFromMergeTree::ReadFromMergeTree( } } + +Pipe ReadFromMergeTree::readFromPoolParallelReplicas( + RangesInDataParts parts_with_range, + Names required_columns, + size_t max_streams, + size_t min_marks_for_concurrent_read, + bool use_uncompressed_cache +) +{ + const auto & client_info = context->getClientInfo(); + auto extension = ParallelReadingExtension + { + .all_callback = all_ranges_callback.value(), + .callback = read_task_callback.value(), + .count_participating_replicas = client_info.count_participating_replicas, + .number_of_current_replica = client_info.number_of_current_replica, + .colums_to_read = required_columns + }; + + /// We have a special logic for local replica. It has to read less data, because in some cases it should + /// merge states of aggregate functions or do some other important stuff other than reading from Disk. + auto is_local_replica = context->getClientInfo().interface == ClientInfo::Interface::LOCAL; + if (!is_local_replica) + min_marks_for_concurrent_read = static_cast(min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier); + + auto pool = std::make_shared( + storage_snapshot, + max_streams, + extension, + parts_with_range, + prewhere_info, + required_columns, + virt_column_names, + min_marks_for_concurrent_read + ); + + Pipes pipes; + const auto & settings = context->getSettingsRef(); + size_t total_rows = parts_with_range.getRowsCountAllParts(); + + for (size_t i = 0; i < max_streams; ++i) + { + auto algorithm = std::make_unique( + i, pool, min_marks_for_concurrent_read, max_block_size, + settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, + data, storage_snapshot, use_uncompressed_cache, + prewhere_info, actions_settings, reader_settings, virt_column_names); + + auto source = std::make_shared(std::move(algorithm)); + + /// Set the approximate number of rows for the first source only + /// In case of parallel processing on replicas do not set approximate rows at all. + /// Because the value will be identical on every replicas and will be accounted + /// multiple times (settings.max_parallel_replicas times more) + if (i == 0 && !client_info.collaborate_with_initiator) + source->addTotalRowsApprox(total_rows); + + pipes.emplace_back(std::move(source)); + + /// Add a special dependency transform which will be connected later with + /// all RemoteSources through a simple scheduler (ResizeProcessor) + if (context->getClientInfo().interface == ClientInfo::Interface::LOCAL) + { + pipes.back().addSimpleTransform([&](const Block & header) -> ProcessorPtr + { + return std::make_shared(header, context->getParallelReplicasGroupUUID()); + }); + } + } + + return Pipe::unitePipes(std::move(pipes)); +} + + Pipe ReadFromMergeTree::readFromPool( RangesInDataParts parts_with_range, Names required_columns, @@ -181,22 +265,25 @@ Pipe ReadFromMergeTree::readFromPool( bool use_uncompressed_cache) { Pipes pipes; - size_t sum_marks = 0; - size_t total_rows = 0; - - for (const auto & part : parts_with_range) - { - sum_marks += part.getMarksCount(); - total_rows += part.getRowsCount(); - } + size_t sum_marks = parts_with_range.getMarksCountAllParts(); + size_t total_rows = parts_with_range.getRowsCountAllParts(); if (query_info.limit > 0 && query_info.limit < total_rows) total_rows = query_info.limit; const auto & settings = context->getSettingsRef(); - const auto & client_info = context->getClientInfo(); MergeTreeReadPool::BackoffSettings backoff_settings(settings); + /// round min_marks_to_read up to nearest multiple of block_size expressed in marks + /// If granularity is adaptive it doesn't make sense + /// Maybe it will make sense to add settings `max_block_size_bytes` + if (max_block_size && !data.canUseAdaptiveGranularity()) + { + size_t fixed_index_granularity = data.getSettings()->index_granularity; + min_marks_for_concurrent_read = (min_marks_for_concurrent_read * fixed_index_granularity + max_block_size - 1) + / max_block_size * max_block_size / fixed_index_granularity; + } + auto pool = std::make_shared( max_streams, sum_marks, @@ -215,34 +302,17 @@ Pipe ReadFromMergeTree::readFromPool( for (size_t i = 0; i < max_streams; ++i) { - std::optional extension; - if (read_task_callback) - { - extension = ParallelReadingExtension - { - .callback = read_task_callback.value(), - .count_participating_replicas = client_info.count_participating_replicas, - .number_of_current_replica = client_info.number_of_current_replica, - .colums_to_read = required_columns - }; - } - auto algorithm = std::make_unique( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, storage_snapshot, use_uncompressed_cache, - prewhere_info, actions_settings, reader_settings, virt_column_names, std::move(extension)); + prewhere_info, actions_settings, reader_settings, virt_column_names); auto source = std::make_shared(std::move(algorithm)); - /// Set the approximate number of rows for the first source only - /// In case of parallel processing on replicas do not set approximate rows at all. - /// Because the value will be identical on every replicas and will be accounted - /// multiple times (settings.max_parallel_replicas times more) - if (i == 0 && !client_info.collaborate_with_initiator) + if (i == 0) source->addTotalRowsApprox(total_rows); - pipes.emplace_back(std::move(source)); } @@ -257,21 +327,9 @@ ProcessorPtr ReadFromMergeTree::createSource( const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, - bool has_limit_below_one_block) + bool has_limit_below_one_block, + MergeTreeInOrderReadPoolParallelReplicasPtr pool) { - const auto & client_info = context->getClientInfo(); - std::optional extension; - if (read_task_callback) - { - extension = ParallelReadingExtension - { - .callback = read_task_callback.value(), - .count_participating_replicas = client_info.count_participating_replicas, - .number_of_current_replica = client_info.number_of_current_replica, - .colums_to_read = required_columns - }; - } - auto total_rows = part.getRowsCount(); if (query_info.limit > 0 && query_info.limit < total_rows) total_rows = query_info.limit; @@ -281,12 +339,12 @@ ProcessorPtr ReadFromMergeTree::createSource( /// In this case we won't set approximate rows, because it will be accounted multiple times. /// Also do not count amount of read rows if we read in order of sorting key, /// because we don't know actual amount of read rows in case when limit is set. - bool set_rows_approx = !extension.has_value() && !reader_settings.read_in_order; + bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order; auto algorithm = std::make_unique( data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes, preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, - actions_settings, reader_settings, virt_column_names, part.part_index_in_query, has_limit_below_one_block, std::move(extension)); + actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block); auto source = std::make_shared(std::move(algorithm)); @@ -301,7 +359,8 @@ Pipe ReadFromMergeTree::readInOrder( Names required_columns, ReadType read_type, bool use_uncompressed_cache, - UInt64 limit) + UInt64 limit, + MergeTreeInOrderReadPoolParallelReplicasPtr pool) { Pipes pipes; /// For reading in order it makes sense to read only @@ -311,8 +370,8 @@ Pipe ReadFromMergeTree::readInOrder( for (const auto & part : parts_with_range) { auto source = read_type == ReadType::InReverseOrder - ? createSource(part, required_columns, use_uncompressed_cache, has_limit_below_one_block) - : createSource(part, required_columns, use_uncompressed_cache, has_limit_below_one_block); + ? createSource(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool) + : createSource(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool); pipes.emplace_back(std::move(source)); } @@ -334,11 +393,14 @@ Pipe ReadFromMergeTree::read( RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache) { + if (read_type == ReadType::ParallelReplicas) + return readFromPoolParallelReplicas(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); + if (read_type == ReadType::Default && max_streams > 1) return readFromPool(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache); - auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, 0); + auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, /*limit */0, /*pool*/nullptr); /// Use ConcatProcessor to concat sources together. /// It is needed to read in parts order (and so in PK order) if single thread is used. @@ -425,6 +487,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); + LOG_TRACE(log, "Spreading mark ranges among streams (default reading)"); + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); if (0 == info.sum_marks) @@ -438,7 +502,9 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams( num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size()); } - return read(std::move(parts_with_ranges), column_names, ReadType::Default, + auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default; + + return read(std::move(parts_with_ranges), column_names, read_type, num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache); } @@ -459,6 +525,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); + LOG_TRACE(log, "Spreading ranges among streams with order"); + PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); Pipes res; @@ -534,7 +602,41 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( const size_t min_marks_per_stream = (info.sum_marks - 1) / requested_num_streams + 1; bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold); - Pipes pipes; + std::vector splitted_parts_and_ranges; + splitted_parts_and_ranges.reserve(requested_num_streams); + + const auto read_type = input_order_info->direction == 1 + ? ReadFromMergeTree::ReadType::InOrder + : ReadFromMergeTree::ReadType::InReverseOrder; + + MergeTreeInOrderReadPoolParallelReplicasPtr pool; + + if (is_parallel_reading_from_replicas) + { + const auto & client_info = context->getClientInfo(); + auto extension = ParallelReadingExtension + { + .all_callback = all_ranges_callback.value(), + .callback = read_task_callback.value(), + .count_participating_replicas = client_info.count_participating_replicas, + .number_of_current_replica = client_info.number_of_current_replica, + .colums_to_read = column_names + }; + + /// We have a special logic for local replica. It has to read less data, because in some cases it should + /// merge states of aggregate functions or do some other important stuff other than reading from Disk. + auto is_local_replica = context->getClientInfo().interface == ClientInfo::Interface::LOCAL; + auto min_marks_for_concurrent_read = info.min_marks_for_concurrent_read; + if (!is_local_replica) + min_marks_for_concurrent_read = static_cast(min_marks_for_concurrent_read * settings.parallel_replicas_single_task_marks_count_multiplier); + + pool = std::make_shared( + parts_with_ranges, + extension, + read_type == ReadFromMergeTree::ReadType::InOrder ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder, + min_marks_for_concurrent_read); + } + for (size_t i = 0; i < requested_num_streams && !parts_with_ranges.empty(); ++i) { @@ -602,12 +704,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part)); } - auto read_type = input_order_info->direction == 1 - ? ReadFromMergeTree::ReadType::InOrder - : ReadFromMergeTree::ReadType::InReverseOrder; + splitted_parts_and_ranges.emplace_back(std::move(new_parts)); + } - pipes.emplace_back(readInOrder(std::move(new_parts), column_names, read_type, - info.use_uncompressed_cache, input_order_info->limit)); + Pipes pipes; + for (auto & item : splitted_parts_and_ranges) + { + pipes.emplace_back(readInOrder(std::move(item), column_names, read_type, + info.use_uncompressed_cache, input_order_info->limit, pool)); } Block pipe_header; @@ -758,7 +862,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 /// we will store lonely parts with level > 0 to use parallel select on them. - std::vector lonely_parts; + RangesInDataParts lonely_parts; size_t sum_marks_in_lonely_parts = 0; for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) @@ -1265,6 +1369,17 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons const auto & input_order_info = query_info.getInputOrderInfo(); + /// Construct a proper coordinator + if (input_order_info && is_parallel_reading_from_replicas && context->getClientInfo().interface == ClientInfo::Interface::LOCAL) + { + assert(context->parallel_reading_coordinator); + auto mode = input_order_info->direction == 1 ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder; + context->parallel_reading_coordinator->setMode(mode); + } + + if (final && is_parallel_reading_from_replicas) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Final modifier is not supported with parallel replicas"); + if (final) { /// Add columns needed to calculate the sorting expression and the sign. @@ -1406,6 +1521,8 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type) return "InOrder"; case ReadFromMergeTree::ReadType::InReverseOrder: return "InReverseOrder"; + case ReadFromMergeTree::ReadType::ParallelReplicas: + return "Parallel"; } UNREACHABLE(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index a3cea2a8afe..8b2eca5e08e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -11,7 +13,7 @@ using PartitionIdToMaxBlock = std::unordered_map; class Pipe; -using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; +using MergeTreeReadTaskCallback = std::function(ParallelReadRequest)>; struct MergeTreeDataSelectSamplingData { @@ -68,6 +70,10 @@ public: /// The same as InOrder, but in reverse order. /// For every part, read ranges and granules from end to begin. Also add ReverseTransform. InReverseOrder, + /// A special type of reading where every replica + /// talks to a remote coordinator (which is located on the initiator node) + /// and who spreads marks and parts across them. + ParallelReplicas, }; struct AnalysisResult @@ -212,10 +218,11 @@ private: Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); - Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit); + Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache); + Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit, MergeTreeInOrderReadPoolParallelReplicasPtr pool); template - ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block); + ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block, MergeTreeInOrderReadPoolParallelReplicasPtr pool); Pipe spreadMarkRangesAmongStreams( RangesInDataParts && parts_with_ranges, @@ -236,6 +243,8 @@ private: ReadFromMergeTree::AnalysisResult getAnalysisResult() const; MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr; + bool is_parallel_reading_from_replicas; + std::optional all_ranges_callback; std::optional read_task_callback; }; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 103f2734b06..9e5ecc791dc 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -9,10 +9,13 @@ #include #include #include +#include #include #include #include +#include "Common/logger_useful.h" #include +#include #include #include #include @@ -180,7 +183,8 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { - auto plan = createLocalPlan(query, header, context, stage, shard.shard_info.shard_num, shard_count, 0, 0, /*coordinator=*/nullptr); + auto plan = createLocalPlan( + query, header, context, stage, shard.shard_info.shard_num, shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), @@ -231,7 +235,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact std::shared_ptr remote_query_executor; remote_query_executor = std::make_shared( - shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage); + shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); remote_query_executor->setPoolMode(PoolMode::GET_MANY); @@ -265,8 +269,9 @@ void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const B ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( + ASTPtr query_ast_, + Cluster::ShardInfo shard_info_, ParallelReplicasReadingCoordinatorPtr coordinator_, - ClusterProxy::SelectStreamFactory::Shard shard_, Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, @@ -276,10 +281,12 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Scalars scalars_, Tables external_tables_, Poco::Logger * log_, - std::shared_ptr storage_limits_) + std::shared_ptr storage_limits_, + UUID uuid_) : ISourceStep(DataStream{.header = std::move(header_)}) + , shard_info(shard_info_) + , query_ast(query_ast_) , coordinator(std::move(coordinator_)) - , shard(std::move(shard_)) , stage(std::move(stage_)) , main_table(std::move(main_table_)) , table_func_ptr(table_func_ptr_) @@ -289,10 +296,11 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , external_tables{external_tables_} , storage_limits(std::move(storage_limits_)) , log(log_) + , uuid(uuid_) { std::vector description; - for (const auto & address : shard.shard_info.all_addresses) + for (const auto & address : shard_info.all_addresses) if (!address.is_local) description.push_back(fmt::format("Replica: {}", address.host_name)); @@ -312,28 +320,46 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder() void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; - const Settings & current_settings = context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - for (size_t replica_num = 0; replica_num < shard.shard_info.getAllNodeCount(); ++replica_num) + size_t all_replicas_count = current_settings.max_parallel_replicas; + if (all_replicas_count > shard_info.all_addresses.size()) { - if (shard.shard_info.all_addresses[replica_num].is_local) + LOG_INFO(&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), + "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "\ + "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, shard_info.all_addresses.size()); + all_replicas_count = shard_info.all_addresses.size(); + } + + /// The requested number of replicas to read from could be less + /// than the total number of replicas in the shard + /// And we have to pick only "remote" ones + /// So, that's why this loop looks like this. + size_t replica_num = 0; + while (pipes.size() != all_replicas_count - 1) + { + if (shard_info.all_addresses[replica_num].is_local) + { + ++replica_num; continue; + } IConnections::ReplicaInfo replica_info { - .all_replicas_count = shard.shard_info.getAllNodeCount(), - .number_of_current_replica = replica_num + .all_replicas_count = all_replicas_count, + /// Replica 0 is threated as local always + .number_of_current_replica = pipes.size() + 1 }; - auto pool = shard.shard_info.per_replica_pools[replica_num]; + auto pool = shard_info.per_replica_pools[replica_num]; assert(pool); auto pool_with_failover = std::make_shared( ConnectionPoolPtrs{pool}, current_settings.load_balancing); addPipeForSingeReplica(pipes, std::move(pool_with_failover), replica_info); + ++replica_num; } auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -355,25 +381,22 @@ void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, s if (stage == QueryProcessingStage::Complete) { - add_totals = shard.query->as().group_by_with_totals; + add_totals = query_ast->as().group_by_with_totals; add_extremes = context->getSettingsRef().extremes; } - String query_string = formattedAST(shard.query); + String query_string = formattedAST(query_ast); - scalars["_shard_num"] - = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; + assert(stage != QueryProcessingStage::Complete); + assert(output_stream); auto remote_query_executor = std::make_shared( - pool, query_string, shard.header, context, throttler, scalars, external_tables, stage, + pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage, RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)}); remote_query_executor->setLogger(log); - if (!table_func_ptr) - remote_query_executor->setMainTable(main_table); - - pipes.emplace_back(createRemoteSourcePipe(std::move(remote_query_executor), add_agg_info, add_totals, add_extremes, async_read)); + pipes.emplace_back(createRemoteSourcePipe(std::move(remote_query_executor), add_agg_info, add_totals, add_extremes, async_read, uuid)); addConvertingActions(pipes.back(), output_stream->header); } diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 60a7cd90f3f..e1979ee1aaa 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -6,6 +6,7 @@ #include #include #include +#include "Core/UUID.h" namespace DB { @@ -45,18 +46,13 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; - StorageID main_table; ASTPtr table_func_ptr; - ContextMutablePtr context; - ThrottlerPtr throttler; Scalars scalars; Tables external_tables; - std::shared_ptr storage_limits; - Poco::Logger * log; UInt32 shard_count; @@ -69,8 +65,9 @@ class ReadFromParallelRemoteReplicasStep : public ISourceStep { public: ReadFromParallelRemoteReplicasStep( + ASTPtr query_ast_, + Cluster::ShardInfo shard_info, ParallelReplicasReadingCoordinatorPtr coordinator_, - ClusterProxy::SelectStreamFactory::Shard shard, Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, @@ -80,7 +77,8 @@ public: Scalars scalars_, Tables external_tables_, Poco::Logger * log_, - std::shared_ptr storage_limits_); + std::shared_ptr storage_limits_, + UUID uuid); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -93,22 +91,20 @@ private: void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr pool, IConnections::ReplicaInfo replica_info); + Cluster::ShardInfo shard_info; + ASTPtr query_ast; ParallelReplicasReadingCoordinatorPtr coordinator; - ClusterProxy::SelectStreamFactory::Shard shard; QueryProcessingStage::Enum stage; - StorageID main_table; ASTPtr table_func_ptr; - ContextMutablePtr context; - ThrottlerPtr throttler; Scalars scalars; Tables external_tables; std::shared_ptr storage_limits; - Poco::Logger * log; + UUID uuid; }; } diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 9f29ad9ad07..69964d569fa 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -8,10 +9,16 @@ namespace DB { -RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_, UUID uuid_) : ISource(executor->getHeader(), false) , add_aggregation_info(add_aggregation_info_), query_executor(std::move(executor)) , async_read(async_read_) + , uuid(uuid_) { /// Add AggregatedChunkInfo if we expect DataTypeAggregateFunction as a result. const auto & sample = getPort().getHeader(); @@ -22,6 +29,18 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation RemoteSource::~RemoteSource() = default; +void RemoteSource::connectToScheduler(InputPort & input_port) +{ + outputs.emplace_back(Block{}, this); + dependency_port = &outputs.back(); + connect(*dependency_port, input_port); +} + +UUID RemoteSource::getParallelReplicasGroupUUID() +{ + return uuid; +} + void RemoteSource::setStorageLimits(const std::shared_ptr & storage_limits_) { /// Remove leaf limits for remote source. @@ -50,8 +69,21 @@ ISource::Status RemoteSource::prepare() if (status == Status::Finished) { query_executor->finish(&read_context); + if (dependency_port) + dependency_port->finish(); is_async_state = false; + + return status; } + + if (status == Status::PortFull) + { + /// Also push empty chunk to dependency to signal that we read data from remote source + /// or answered to the incoming request from parallel replica + if (dependency_port && !dependency_port->isFinished() && dependency_port->canPush()) + dependency_port->push(Chunk()); + } + return status; } @@ -88,19 +120,29 @@ std::optional RemoteSource::tryGenerate() if (async_read) { auto res = query_executor->read(read_context); - if (std::holds_alternative(res)) + + if (res.getType() == RemoteQueryExecutor::ReadResult::Type::Nothing) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an empty packet from the RemoteQueryExecutor. This is a bug"); + + if (res.getType() == RemoteQueryExecutor::ReadResult::Type::FileDescriptor) { - fd = std::get(res); + fd = res.getFileDescriptor(); is_async_state = true; return Chunk(); } + if (res.getType() == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken) + { + is_async_state = false; + return Chunk(); + } + is_async_state = false; - block = std::get(std::move(res)); + block = res.getBlock(); } else - block = query_executor->read(); + block = query_executor->readBlock(); if (!block) { @@ -180,9 +222,9 @@ Chunk RemoteExtremesSource::generate() Pipe createRemoteSourcePipe( RemoteQueryExecutorPtr query_executor, - bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read) + bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read, UUID uuid) { - Pipe pipe(std::make_shared(query_executor, add_aggregation_info, async_read)); + Pipe pipe(std::make_shared(query_executor, add_aggregation_info, async_read, uuid)); if (add_totals) pipe.addTotalsSource(std::make_shared(query_executor)); diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index f415b91aae0..8fe0114ab6f 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -3,6 +3,7 @@ #include #include #include +#include "Core/UUID.h" #include namespace DB @@ -14,20 +15,24 @@ using RemoteQueryExecutorPtr = std::shared_ptr; class RemoteQueryExecutorReadContext; /// Source from RemoteQueryExecutor. Executes remote query and returns query result chunks. -class RemoteSource : public ISource +class RemoteSource final : public ISource { public: /// Flag add_aggregation_info tells if AggregatedChunkInfo should be added to result chunk. /// AggregatedChunkInfo stores the bucket number used for two-level aggregation. /// This flag should be typically enabled for queries with GROUP BY which are executed till WithMergeableState. - RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_); + RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_, UUID uuid = UUIDHelpers::Nil); ~RemoteSource() override; Status prepare() override; String getName() const override { return "Remote"; } + void connectToScheduler(InputPort & input_port); + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); } + UUID getParallelReplicasGroupUUID(); + /// Stop reading from stream if output port is finished. void onUpdatePorts() override; @@ -46,9 +51,12 @@ private: RemoteQueryExecutorPtr query_executor; RowsBeforeLimitCounterPtr rows_before_limit; + OutputPort * dependency_port{nullptr}; + const bool async_read; bool is_async_state = false; std::unique_ptr read_context; + UUID uuid; int fd = -1; }; @@ -87,6 +95,6 @@ private: /// Create pipe with remote sources. Pipe createRemoteSourcePipe( RemoteQueryExecutorPtr query_executor, - bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read); + bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read, UUID uuid = UUIDHelpers::Nil); } diff --git a/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.cpp b/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.cpp new file mode 100644 index 00000000000..295eddb206d --- /dev/null +++ b/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.cpp @@ -0,0 +1,103 @@ +#include + +#include +#include "Processors/Port.h" + +namespace DB +{ + +ReadFromMergeTreeDependencyTransform::ReadFromMergeTreeDependencyTransform(const Block & header, UUID uuid_) + : IProcessor(InputPorts(1, header), OutputPorts(1, header)) + , uuid(uuid_) + , data_port(&inputs.front()) +{ +} + +void ReadFromMergeTreeDependencyTransform::connectToScheduler(OutputPort & output_port) +{ + inputs.emplace_back(Block{}, this); + dependency_port = &inputs.back(); + connect(output_port, *dependency_port); +} + +UUID ReadFromMergeTreeDependencyTransform::getParallelReplicasGroupUUID() +{ + return uuid; +} + +IProcessor::Status ReadFromMergeTreeDependencyTransform::prepare() +{ + Status status = Status::Ready; + + while (status == Status::Ready) + { + status = !has_data ? prepareConsume() + : prepareGenerate(); + } + + return status; +} + +IProcessor::Status ReadFromMergeTreeDependencyTransform::prepareConsume() +{ + auto & output_port = getOutputPort(); + + /// Check all outputs are finished or ready to get data. + if (output_port.isFinished()) + { + data_port->close(); + dependency_port->close(); + return Status::Finished; + } + + /// Try get chunk from input. + if (data_port->isFinished()) + { + if (dependency_port->hasData()) + dependency_port->pull(true); + dependency_port->close(); + output_port.finish(); + return Status::Finished; + } + + if (!dependency_port->isFinished()) + { + dependency_port->setNeeded(); + if (!dependency_port->hasData()) + return Status::NeedData; + } + + data_port->setNeeded(); + if (!data_port->hasData()) + return Status::NeedData; + + if (!dependency_port->isFinished()) + dependency_port->pull(); + + chunk = data_port->pull(); + has_data = true; + + return Status::Ready; +} + +IProcessor::Status ReadFromMergeTreeDependencyTransform::prepareGenerate() +{ + auto & output_port = getOutputPort(); + if (!output_port.isFinished() && output_port.canPush()) + { + output_port.push(std::move(chunk)); + has_data = false; + return Status::Ready; + } + + if (output_port.isFinished()) + { + data_port->close(); + dependency_port->close(); + return Status::Finished; + } + + return Status::PortFull; +} + +} diff --git a/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.h b/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.h new file mode 100644 index 00000000000..929841e7ce0 --- /dev/null +++ b/src/Processors/Transforms/ReadFromMergeTreeDependencyTransform.h @@ -0,0 +1,48 @@ +#pragma once +#include + +namespace DB +{ + +class RemoteQueryExecutor; +using RemoteQueryExecutorPtr = std::shared_ptr; + +/// A tiny class which is used for reading with multiple replicas in parallel. +/// Motivation is that we don't have a full control on how +/// processors are scheduled across threads and there could be a situation +/// when all available threads will read from local replica and will just +/// forget about remote replicas existence. That is not what we want. +/// For parallel replicas we have to constantly answer to incoming requests +/// with a set of marks to read. +/// With the help of this class, we explicitly connect a "local" source with +/// all the remote ones. And thus achieve fairness somehow. +class ReadFromMergeTreeDependencyTransform : public IProcessor +{ +public: + ReadFromMergeTreeDependencyTransform(const Block & header, UUID uuid_); + + String getName() const override { return "ReadFromMergeTreeDependency"; } + Status prepare() override; + + InputPort & getInputPort() { assert(data_port); return *data_port; } + InputPort & getDependencyPort() { assert(dependency_port); return *dependency_port; } + OutputPort & getOutputPort() { return outputs.front(); } + + UUID getParallelReplicasGroupUUID(); + + void connectToScheduler(OutputPort & output_port); +private: + bool has_data{false}; + Chunk chunk; + + UUID uuid; + + InputPort * data_port{nullptr}; + InputPort * dependency_port{nullptr}; + + Status prepareGenerate(); + Status prepareConsume(); +}; + + +} diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 2b61bfe7573..09931e38578 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -102,6 +102,8 @@ public: /// Get processors from Pipe without destroying pipe (used for EXPLAIN to keep QueryPlan). const Processors & getProcessors() const { return *processors; } + std::shared_ptr getProcessorsPtr() { return processors; } + private: /// Header is common for all output below. Block header; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 483447d1e4d..07adc6b0b3a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -1,34 +1,35 @@ -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include "Core/UUID.h" +#include #include #include #include #include -#include -#include +#include #include -#include -#include #include +#include +#include +#include +#include +#include +#include #include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -620,6 +621,65 @@ void QueryPipelineBuilder::setProgressCallback(ProgressCallback callback) progress_callback = callback; } +void QueryPipelineBuilder::connectDependencies() +{ + /** + * This is needed because among all RemoteSources there could be + * one or several that don't belong to the parallel replicas reading process. + * It could happen for example if we read through distributed table + prefer_localhost_replica=1 + parallel replicas + * SELECT * FROM remote('127.0.0.{1,2}', table.merge_tree) + * Will generate a local pipeline and a remote source. For local pipeline because of parallel replicas we will create + * several processors to read and several remote sources. + */ + std::set all_parallel_replicas_groups; + for (auto & processor : *pipe.getProcessorsPtr()) + { + if (auto * remote_dependency = typeid_cast(processor.get()); remote_dependency) + if (auto uuid = remote_dependency->getParallelReplicasGroupUUID(); uuid != UUIDHelpers::Nil) + all_parallel_replicas_groups.insert(uuid); + if (auto * merge_tree_dependency = typeid_cast(processor.get()); merge_tree_dependency) + if (auto uuid = merge_tree_dependency->getParallelReplicasGroupUUID(); uuid != UUIDHelpers::Nil) + all_parallel_replicas_groups.insert(uuid); + } + + for (const auto & group_id : all_parallel_replicas_groups) + { + std::vector input_dependencies; + std::vector output_dependencies; + + for (auto & processor : *pipe.getProcessorsPtr()) + { + if (auto * remote_dependency = typeid_cast(processor.get()); remote_dependency) + if (auto uuid = remote_dependency->getParallelReplicasGroupUUID(); uuid == group_id) + input_dependencies.emplace_back(remote_dependency); + if (auto * merge_tree_dependency = typeid_cast(processor.get()); merge_tree_dependency) + if (auto uuid = merge_tree_dependency->getParallelReplicasGroupUUID(); uuid == group_id) + output_dependencies.emplace_back(merge_tree_dependency); + } + + if (input_dependencies.empty() || output_dependencies.empty()) + continue; + + auto input_dependency_iter = input_dependencies.begin(); + auto output_dependency_iter = output_dependencies.begin(); + auto scheduler = std::make_shared(Block{}, input_dependencies.size(), output_dependencies.size()); + + for (auto & scheduler_input : scheduler->getInputs()) + { + (*input_dependency_iter)->connectToScheduler(scheduler_input); + ++input_dependency_iter; + } + + for (auto & scheduler_output : scheduler->getOutputs()) + { + (*output_dependency_iter)->connectToScheduler(scheduler_output); + ++output_dependency_iter; + } + + pipe.getProcessorsPtr()->emplace_back(std::move(scheduler)); + } +} + PipelineExecutorPtr QueryPipelineBuilder::execute() { if (!isCompleted()) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 5a0694100eb..0a102d186ca 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -140,6 +140,12 @@ public: void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context); + /// Finds all processors for reading from MergeTree + /// And explicitly connects them with all RemoteSources + /// using a ResizeProcessor. This is needed not to let + /// the RemoteSource to starve for CPU time + void connectDependencies(); + PipelineExecutorPtr execute(); size_t getNumStreams() const { return pipe.numOutputPorts(); } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 961d8129d29..033907e9e2b 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -259,48 +259,62 @@ void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind) sendExternalTables(); } -Block RemoteQueryExecutor::read() + +Block RemoteQueryExecutor::readBlock() +{ + while (true) + { + auto res = read(); + + if (res.getType() == ReadResult::Type::Data) + return res.getBlock(); + } +} + + +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read() { if (!sent_query) { sendQuery(); if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size())) - return {}; + return ReadResult(Block()); } while (true) { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) - return Block(); + return ReadResult(Block()); - Packet packet = connections->receivePacket(); + auto packet = connections->receivePacket(); + auto anything = processPacket(std::move(packet)); - if (auto block = processPacket(std::move(packet))) - return *block; - else if (got_duplicated_part_uuids) - return std::get(restartQueryWithoutDuplicatedUUIDs()); + if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken) + return anything; + + if (got_duplicated_part_uuids) + return restartQueryWithoutDuplicatedUUIDs(); } } -std::variant RemoteQueryExecutor::read(std::unique_ptr & read_context [[maybe_unused]]) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read(std::unique_ptr & read_context [[maybe_unused]]) { - #if defined(OS_LINUX) if (!sent_query) { sendQuery(); if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size())) - return Block(); + return ReadResult(Block()); } if (!read_context || resent_query) { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) - return Block(); + return ReadResult(Block()); read_context = std::make_unique(*connections); } @@ -308,12 +322,12 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr do { if (!read_context->resumeRoutine()) - return Block(); + return ReadResult(Block()); if (read_context->is_read_in_progress.load(std::memory_order_relaxed)) { read_context->setTimer(); - return read_context->epoll.getFileDescriptor(); + return ReadResult(read_context->epoll.getFileDescriptor()); } else { @@ -321,11 +335,14 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr /// to avoid the race between cancel() thread and read() thread. /// (since cancel() thread will steal the fiber and may update the packet). if (was_cancelled) - return Block(); + return ReadResult(Block()); - if (auto data = processPacket(std::move(read_context->packet))) - return std::move(*data); - else if (got_duplicated_part_uuids) + auto anything = processPacket(std::move(read_context->packet)); + + if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken) + return anything; + + if (got_duplicated_part_uuids) return restartQueryWithoutDuplicatedUUIDs(&read_context); } } @@ -336,7 +353,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr } -std::variant RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context) { /// Cancel previous query and disconnect before retry. cancel(read_context); @@ -360,13 +377,18 @@ std::variant RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate uuids while processing query"); } -std::optional RemoteQueryExecutor::processPacket(Packet packet) +RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet) { switch (packet.type) { case Protocol::Server::MergeTreeReadTaskRequest: processMergeTreeReadTaskRequest(packet.request); - break; + return ReadResult(ReadResult::Type::ParallelReplicasToken); + + case Protocol::Server::MergeTreeAllRangesAnnounecement: + processMergeTreeInitialReadAnnounecement(packet.announcement); + return ReadResult(ReadResult::Type::ParallelReplicasToken); + case Protocol::Server::ReadTaskRequest: processReadTaskRequest(); break; @@ -379,7 +401,7 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) /// We can actually return it, and the first call to RemoteQueryExecutor::read /// will return earlier. We should consider doing it. if (packet.block && (packet.block.rows() > 0)) - return adaptBlockStructure(packet.block, header); + return ReadResult(adaptBlockStructure(packet.block, header)); break; /// If the block is empty - we will receive other packets before EndOfStream. case Protocol::Server::Exception: @@ -391,7 +413,8 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) if (!connections->hasActiveConnections()) { finished = true; - return Block(); + /// TODO: Replace with Type::Finished + return ReadResult(Block{}); } break; @@ -446,7 +469,7 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) connections->dumpAddresses()); } - return {}; + return ReadResult(ReadResult::Type::Nothing); } bool RemoteQueryExecutor::setPartUUIDs(const std::vector & uuids) @@ -471,7 +494,7 @@ void RemoteQueryExecutor::processReadTaskRequest() connections->sendReadTaskResponse(response); } -void RemoteQueryExecutor::processMergeTreeReadTaskRequest(PartitionReadRequest request) +void RemoteQueryExecutor::processMergeTreeReadTaskRequest(ParallelReadRequest request) { if (!parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); @@ -480,6 +503,14 @@ void RemoteQueryExecutor::processMergeTreeReadTaskRequest(PartitionReadRequest r connections->sendMergeTreeReadTaskResponse(response); } +void RemoteQueryExecutor::processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement) +{ + if (!parallel_reading_coordinator) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); + + parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); +} + void RemoteQueryExecutor::finish(std::unique_ptr * read_context) { /** If one of: diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 8b8f21a3ae4..c67a45c7275 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -94,12 +95,60 @@ public: /// Query is resent to a replica, the query itself can be modified. std::atomic resent_query { false }; + struct ReadResult + { + enum class Type : uint8_t + { + Data, + ParallelReplicasToken, + FileDescriptor, + Finished, + Nothing + }; + + explicit ReadResult(Block block_) + : type(Type::Data) + , block(std::move(block_)) + {} + + explicit ReadResult(int fd_) + : type(Type::FileDescriptor) + , fd(fd_) + {} + + explicit ReadResult(Type type_) + : type(type_) + { + assert(type != Type::Data && type != Type::FileDescriptor); + } + + Type getType() const { return type; } + + Block getBlock() + { + chassert(type == Type::Data); + return std::move(block); + } + + int getFileDescriptor() const + { + chassert(type == Type::FileDescriptor); + return fd; + } + + Type type; + Block block; + int fd{-1}; + }; + /// Read next block of data. Returns empty block if query is finished. - Block read(); + Block readBlock(); + + ReadResult read(); /// Async variant of read. Returns ready block or file descriptor which may be used for polling. /// ReadContext is an internal read state. Pass empty ptr first time, reuse created one for every call. - std::variant read(std::unique_ptr & read_context); + ReadResult read(std::unique_ptr & read_context); /// Receive all remain packets and finish query. /// It should be cancelled after read returned empty block. @@ -231,11 +280,12 @@ private: void processReadTaskRequest(); - void processMergeTreeReadTaskRequest(PartitionReadRequest request); + void processMergeTreeReadTaskRequest(ParallelReadRequest request); + void processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement); /// Cancel query and restart it with info about duplicate UUIDs /// only for `allow_experimental_query_deduplication`. - std::variant restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context = nullptr); + ReadResult restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context = nullptr); /// If wasn't sent yet, send request to cancel all connections to replicas void tryCancel(const char * reason, std::unique_ptr * read_context); @@ -247,11 +297,10 @@ private: bool hasThrownException() const; /// Process packet for read and return data block if possible. - std::optional processPacket(Packet packet); + ReadResult processPacket(Packet packet); /// Reads packet by packet Block readPackets(); - }; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a48a3bb1ed6..28377edf8ca 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -54,6 +54,7 @@ #include #include "Core/Protocol.h" +#include "Storages/MergeTree/RequestResponse.h" #include "TCPHandler.h" #include "config_version.h" @@ -363,7 +364,17 @@ void TCPHandler::runImpl() return receiveReadTaskResponseAssumeLocked(); }); - query_context->setMergeTreeReadTaskCallback([this](PartitionReadRequest request) -> std::optional + query_context->setMergeTreeAllRangesCallback([this](InitialAllRangesAnnouncement announcement) + { + std::lock_guard lock(task_callback_mutex); + + if (state.is_cancelled) + return; + + sendMergeTreeAllRangesAnnounecementAssumeLocked(announcement); + }); + + query_context->setMergeTreeReadTaskCallback([this](ParallelReadRequest request) -> std::optional { std::lock_guard lock(task_callback_mutex); @@ -920,7 +931,15 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() } -void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request) +void TCPHandler::sendMergeTreeAllRangesAnnounecementAssumeLocked(InitialAllRangesAnnouncement announcement) +{ + writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnounecement, *out); + announcement.serialize(*out); + out->next(); +} + + +void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request) { writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); request.serialize(*out); @@ -1348,7 +1367,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() } -std::optional TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked() +std::optional TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked() { UInt64 packet_type = 0; readVarUInt(packet_type, *in); @@ -1371,7 +1390,7 @@ std::optional TCPHandler::receivePartitionMergeTreeReadTa Protocol::Client::toString(packet_type)); } } - PartitionReadResponse response; + ParallelReadResponse response; response.deserialize(*in); return response; } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 0b296aaef4e..f06b0b060b3 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -21,6 +21,7 @@ #include "IServer.h" #include "Server/TCPProtocolStackData.h" +#include "Storages/MergeTree/RequestResponse.h" #include "base/types.h" @@ -220,7 +221,7 @@ private: void receiveQuery(); void receiveIgnoredPartUUIDs(); String receiveReadTaskResponseAssumeLocked(); - std::optional receivePartitionMergeTreeReadTaskResponseAssumeLocked(); + std::optional receivePartitionMergeTreeReadTaskResponseAssumeLocked(); bool receiveData(bool scalar); bool readDataNext(); void readData(); @@ -253,7 +254,8 @@ private: void sendEndOfStream(); void sendPartUUIDs(); void sendReadTaskRequestAssumeLocked(); - void sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request); + void sendMergeTreeAllRangesAnnounecementAssumeLocked(InitialAllRangesAnnouncement announcement); + void sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request); void sendProfileInfo(const ProfileInfo & info); void sendTotals(const Block & totals); void sendExtremes(const Block & extremes); diff --git a/src/Storages/MergeTree/IntersectionsIndexes.h b/src/Storages/MergeTree/IntersectionsIndexes.h index 68ccbc4a0b1..d9445f446ce 100644 --- a/src/Storages/MergeTree/IntersectionsIndexes.h +++ b/src/Storages/MergeTree/IntersectionsIndexes.h @@ -136,7 +136,7 @@ struct HalfIntervals MarkRanges convertToMarkRangesFinal() { MarkRanges result; - std::move(intervals.begin(), intervals.end(), std::back_inserter(result)); + std::copy(intervals.begin(), intervals.end(), std::back_inserter(result)); return result; } diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index 0eea0e5afd1..c6c197919f4 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -1,5 +1,8 @@ #include "MarkRange.h" +#include +#include + namespace DB { @@ -8,6 +11,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +size_t MarkRange::getNumberOfMarks() const +{ + return end - begin; +} + bool MarkRange::operator==(const MarkRange & rhs) const { return begin == rhs.begin && end == rhs.end; @@ -65,4 +73,41 @@ void assertSortedAndNonIntersecting(const MarkRanges & ranges) toString(ranges)); } +size_t MarkRanges::getNumberOfMarks() const +{ + size_t result = 0; + for (const auto & mark : *this) + result += mark.getNumberOfMarks(); + return result; +} + +void MarkRanges::serialize(WriteBuffer & out) const +{ + writeIntBinary(this->size(), out); + + for (const auto & [begin, end] : *this) + { + writeIntBinary(begin, out); + writeIntBinary(end, out); + } +} + +String MarkRanges::describe() const +{ + return fmt::format("Size: {}, Data: {}", this->size(), fmt::join(*this, ",")); +} + +void MarkRanges::deserialize(ReadBuffer & in) +{ + size_t size = 0; + readIntBinary(size, in); + + this->resize(size); + for (size_t i = 0; i < size; ++i) + { + readIntBinary((*this)[i].begin, in); + readIntBinary((*this)[i].end, in); + } +} + } diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 076fc7dfea2..d1f4e1a4b45 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -4,7 +4,11 @@ #include #include +#include +#include + #include +#include namespace DB { @@ -21,12 +25,22 @@ struct MarkRange MarkRange() = default; MarkRange(const size_t begin_, const size_t end_) : begin{begin_}, end{end_} {} - bool operator==(const MarkRange & rhs) const; + size_t getNumberOfMarks() const; + bool operator==(const MarkRange & rhs) const; bool operator<(const MarkRange & rhs) const; }; -using MarkRanges = std::deque; +struct MarkRanges : public std::deque +{ + using std::deque::deque; + + size_t getNumberOfMarks() const; + + void serialize(WriteBuffer & out) const; + String describe() const; + void deserialize(ReadBuffer & in); +}; /** Get max range.end from ranges. */ @@ -37,3 +51,26 @@ std::string toString(const MarkRanges & ranges); void assertSortedAndNonIntersecting(const MarkRanges & ranges); } + + +template <> +struct fmt::formatter +{ + constexpr static auto parse(format_parse_context & ctx) + { + const auto * it = ctx.begin(); + const auto * end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("invalid format"); + + return it; + } + + template + auto format(const DB::MarkRange & range, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end)); + } +}; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index e2997df3bb0..db32baadbc6 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -49,8 +49,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, - const Names & virt_column_names_, - std::optional extension_) + const Names & virt_column_names_) : storage(storage_) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) @@ -62,7 +61,6 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( , use_uncompressed_cache(use_uncompressed_cache_) , virt_column_names(virt_column_names_) , partition_value_type(storage.getPartitionValueType()) - , extension(extension_) { header_without_const_virtual_columns = applyPrewhereActions(std::move(header), prewhere_info); size_t non_const_columns_offset = header_without_const_virtual_columns.columns(); @@ -115,86 +113,11 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( bool IMergeTreeSelectAlgorithm::getNewTask() { - /// No parallel reading feature - if (!extension.has_value()) + if (getNewTaskImpl()) { - if (getNewTaskImpl()) - { - finalizeNewTask(); - return true; - } - return false; - } - return getNewTaskParallelReading(); -} - - -bool IMergeTreeSelectAlgorithm::getNewTaskParallelReading() -{ - if (getTaskFromBuffer()) + finalizeNewTask(); return true; - - if (no_more_tasks) - return getDelayedTasks(); - - while (true) - { - /// The end of execution. No task. - if (!getNewTaskImpl()) - { - no_more_tasks = true; - return getDelayedTasks(); - } - - splitCurrentTaskRangesAndFillBuffer(); - - if (getTaskFromBuffer()) - return true; } -} - - -bool IMergeTreeSelectAlgorithm::getTaskFromBuffer() -{ - while (!buffered_ranges.empty()) - { - auto ranges = std::move(buffered_ranges.front()); - buffered_ranges.pop_front(); - - assert(!ranges.empty()); - - auto res = performRequestToCoordinator(ranges, /*delayed=*/false); - - if (Status::Accepted == res) - return true; - - /// To avoid any possibility of ignoring cancellation, exception will be thrown. - if (Status::Cancelled == res) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query had been cancelled"); - } - return false; -} - - -bool IMergeTreeSelectAlgorithm::getDelayedTasks() -{ - while (!delayed_tasks.empty()) - { - task = std::move(delayed_tasks.front()); - delayed_tasks.pop_front(); - - assert(!task->mark_ranges.empty()); - - auto res = performRequestToCoordinator(task->mark_ranges, /*delayed=*/true); - - if (Status::Accepted == res) - return true; - - if (Status::Cancelled == res) - break; - } - - finish(); return false; } @@ -688,170 +611,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getSizeP } -IMergeTreeSelectAlgorithm::Status IMergeTreeSelectAlgorithm::performRequestToCoordinator(MarkRanges requested_ranges, bool delayed) -{ - String partition_id = task->data_part->info.partition_id; - String part_name; - String projection_name; - - if (task->data_part->isProjectionPart()) - { - part_name = task->data_part->getParentPart()->name; - projection_name = task->data_part->name; - } - else - { - part_name = task->data_part->name; - } - - PartBlockRange block_range - { - .begin = task->data_part->info.min_block, - .end = task->data_part->info.max_block - }; - - PartitionReadRequest request - { - .partition_id = std::move(partition_id), - .part_name = std::move(part_name), - .projection_name = std::move(projection_name), - .block_range = std::move(block_range), - .mark_ranges = std::move(requested_ranges) - }; - String request_description = request.toString(); - - /// Consistent hashing won't work with reading in order, because at the end of the execution - /// we could possibly seek back - if (!delayed && canUseConsistentHashingForParallelReading()) - { - const auto hash = request.getConsistentHash(extension->count_participating_replicas); - if (hash != extension->number_of_current_replica) - { - auto delayed_task = std::make_unique(*task); // Create a copy - delayed_task->mark_ranges = std::move(request.mark_ranges); - delayed_tasks.emplace_back(std::move(delayed_task)); - LOG_TRACE(log, "Request delayed by hash: {}", request_description); - return Status::Denied; - } - } - - auto optional_response = extension.value().callback(std::move(request)); - - if (!optional_response.has_value()) - { - LOG_TRACE(log, "Request cancelled: {}", request_description); - return Status::Cancelled; - } - - auto response = optional_response.value(); - - task->mark_ranges = std::move(response.mark_ranges); - - if (response.denied || task->mark_ranges.empty()) - { - LOG_TRACE(log, "Request rejected: {}", request_description); - return Status::Denied; - } - - finalizeNewTask(); - - LOG_TRACE(log, "Request accepted: {}", request_description); - return Status::Accepted; -} - - -size_t IMergeTreeSelectAlgorithm::estimateMaxBatchSizeForHugeRanges() -{ - /// This is an empirical number and it is so, - /// because we have an adaptive granularity by default. - const size_t average_granule_size_bytes = 1024 * 1024 * 10; // 10 MiB - - /// We want to have one RTT per one gigabyte of data read from disk - /// this could be configurable. - const size_t max_size_for_one_request = 1024 * 1024 * 1024; // 1 GiB - - size_t sum_average_marks_size = 0; - /// getColumnSize is not fully implemented for compact parts - if (task->data_part->getType() == IMergeTreeDataPart::Type::Compact) - { - sum_average_marks_size = average_granule_size_bytes; - } - else - { - for (const auto & name : extension->colums_to_read) - { - auto size = task->data_part->getColumnSize(name); - - assert(size.marks != 0); - sum_average_marks_size += size.data_uncompressed / size.marks; - } - } - - if (sum_average_marks_size == 0) - sum_average_marks_size = average_granule_size_bytes; // 10 MiB - - LOG_TEST(log, "Reading from {} part, average mark size is {}", - task->data_part->getTypeName(), sum_average_marks_size); - - return max_size_for_one_request / sum_average_marks_size; -} - -void IMergeTreeSelectAlgorithm::splitCurrentTaskRangesAndFillBuffer() -{ - const size_t max_batch_size = estimateMaxBatchSizeForHugeRanges(); - - size_t current_batch_size = 0; - buffered_ranges.emplace_back(); - - for (const auto & range : task->mark_ranges) - { - auto expand_if_needed = [&] - { - if (current_batch_size > max_batch_size) - { - buffered_ranges.emplace_back(); - current_batch_size = 0; - } - }; - - expand_if_needed(); - - if (range.end - range.begin < max_batch_size) - { - buffered_ranges.back().push_back(range); - current_batch_size += range.end - range.begin; - continue; - } - - auto current_begin = range.begin; - auto current_end = range.begin + max_batch_size; - - while (current_end < range.end) - { - auto current_range = MarkRange{current_begin, current_end}; - buffered_ranges.back().push_back(current_range); - current_batch_size += current_end - current_begin; - - current_begin = current_end; - current_end = current_end + max_batch_size; - - expand_if_needed(); - } - - if (range.end - current_begin > 0) - { - auto current_range = MarkRange{current_begin, range.end}; - buffered_ranges.back().push_back(current_range); - current_batch_size += range.end - current_begin; - - expand_if_needed(); - } - } - - if (buffered_ranges.back().empty()) - buffered_ranges.pop_back(); -} - IMergeTreeSelectAlgorithm::~IMergeTreeSelectAlgorithm() = default; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 3615127ea34..77d2a383e28 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -24,6 +24,7 @@ struct ChunkAndProgress struct ParallelReadingExtension { + MergeTreeAllRangesCallback all_callback; MergeTreeReadTaskCallback callback; size_t count_participating_replicas{0}; size_t number_of_current_replica{0}; @@ -48,8 +49,7 @@ public: UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, - const Names & virt_column_names_ = {}, - std::optional extension_ = {}); + const Names & virt_column_names_ = {}); virtual ~IMergeTreeSelectAlgorithm(); @@ -90,8 +90,6 @@ protected: size_t estimateMaxBatchSizeForHugeRanges(); - virtual bool canUseConsistentHashingForParallelReading() { return false; } - /// Closes readers and unlock part locks virtual void finish() = 0; @@ -164,11 +162,6 @@ protected: MergeTreeReadTaskPtr task; - std::optional extension; - bool no_more_tasks{false}; - std::deque delayed_tasks; - std::deque buffered_ranges; - /// This setting is used in base algorithm only to additionally limit the number of granules to read. /// It is changed in ctor of MergeTreeThreadSelectAlgorithm. /// @@ -186,44 +179,8 @@ private: std::atomic is_cancelled{false}; - enum class Status - { - Accepted, - Cancelled, - Denied - }; - - /// Calls getNewTaskImpl() to get new task, then performs a request to a coordinator - /// The coordinator may modify the set of ranges to read from a part or could - /// deny the whole request. In the latter case it creates new task and retries. - /// Then it calls finalizeNewTask() to create readers for a task if it is needed. bool getNewTask(); - bool getNewTaskParallelReading(); - /// After PK analysis the range of marks could be extremely big - /// We divide this range to a set smaller consecutive ranges - /// Then, depending on the type of reading (concurrent, in order or in reverse order) - /// we can calculate a consistent hash function with the number of buckets equal to - /// the number of replicas involved. And after that we can throw away some ranges with - /// hash not equals to the number of the current replica. - bool getTaskFromBuffer(); - - /// But we can't throw that ranges completely, because if we have different sets of parts - /// on replicas (have merged part on one, but not on another), then such a situation is possible - /// - Coordinator allows to read from a big merged part, but this part is present only on one replica. - /// And that replica calculates consistent hash and throws away some ranges - /// - Coordinator denies other replicas to read from another parts (source parts for that big one) - /// At the end, the result of the query is wrong, because we didn't read all the data. - /// So, we have to remember parts and mark ranges with hash different then current replica number. - /// An we have to ask the coordinator about its permission to read from that "delayed" parts. - /// It won't work with reading in order or reading in reverse order, because we can possibly seek back. - bool getDelayedTasks(); - - /// It will form a request to coordinator and - /// then reinitialize the mark ranges of this->task object - Status performRequestToCoordinator(MarkRanges requested_ranges, bool delayed); - - void splitCurrentTaskRangesAndFillBuffer(); static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info); }; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 6bd8cc60979..e300723b37b 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -135,10 +135,9 @@ NameSet injectRequiredColumns( MergeTreeReadTask::MergeTreeReadTask( - const MergeTreeData::DataPartPtr & data_part_, + const DataPartPtr & data_part_, const MarkRanges & mark_ranges_, size_t part_index_in_query_, - const Names & ordered_names_, const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, bool remove_prewhere_column_, @@ -146,7 +145,6 @@ MergeTreeReadTask::MergeTreeReadTask( : data_part{data_part_} , mark_ranges{mark_ranges_} , part_index_in_query{part_index_in_query_} - , ordered_names{ordered_names_} , column_name_set{column_name_set_} , task_columns{task_columns_} , remove_prewhere_column{remove_prewhere_column_} @@ -156,7 +154,7 @@ MergeTreeReadTask::MergeTreeReadTask( MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor( - const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block) + const DataPartPtr & data_part_, const Names & columns, const Block & sample_block) : data_part(data_part_) { number_of_rows_in_part = data_part->rows_count; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 72cdbc562ee..162b15b6388 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -17,6 +18,9 @@ class IMergeTreeDataPartInfoForReader; using MergeTreeReadTaskPtr = std::unique_ptr; using MergeTreeBlockSizePredictorPtr = std::shared_ptr; +class IMergeTreeDataPart; +using DataPartPtr = std::shared_ptr; + /** If some of the requested columns are not in the part, * then find out which columns may need to be read further, @@ -44,13 +48,11 @@ struct MergeTreeReadTaskColumns struct MergeTreeReadTask { /// data part which should be read while performing this task - MergeTreeData::DataPartPtr data_part; + DataPartPtr data_part; /// Ranges to read from `data_part`. MarkRanges mark_ranges; /// for virtual `part_index` virtual column size_t part_index_in_query; - /// ordered list of column names used in this query, allows returning blocks with consistent ordering - const Names & ordered_names; /// used to determine whether column should be filtered during PREWHERE or WHERE const NameSet & column_name_set; /// column names to read during PREWHERE and WHERE @@ -68,10 +70,9 @@ struct MergeTreeReadTask bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); } MergeTreeReadTask( - const MergeTreeData::DataPartPtr & data_part_, + const DataPartPtr & data_part_, const MarkRanges & mark_ranges_, size_t part_index_in_query_, - const Names & ordered_names_, const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, bool remove_prewhere_column_, @@ -88,7 +89,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( struct MergeTreeBlockSizePredictor { - MergeTreeBlockSizePredictor(const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block); + MergeTreeBlockSizePredictor(const DataPartPtr & data_part_, const Names & columns, const Block & sample_block); /// Reset some values for correct statistics calculating void startBlock(); @@ -137,7 +138,7 @@ struct MergeTreeBlockSizePredictor protected: - MergeTreeData::DataPartPtr data_part; + DataPartPtr data_part; struct ColumnInfo { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d1b7e2eaf5b..0869ef8c262 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,36 +1,48 @@ #include "Storages/MergeTree/MergeTreeDataPartBuilder.h" #include +#include #include #include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include #include #include -#include #include +#include +#include #include #include -#include #include #include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -40,29 +52,19 @@ #include #include #include +#include +#include #include +#include +#include #include #include #include #include #include -#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include @@ -82,6 +84,7 @@ #include #include +#include template <> struct fmt::formatter : fmt::formatter @@ -6807,6 +6810,14 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const { + if (query_context->getClientInfo().collaborate_with_initiator) + return QueryProcessingStage::Enum::FetchColumns; + + if (query_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas + && !query_context->getClientInfo().collaborate_with_initiator + && to_stage >= QueryProcessingStage::WithMergeableState) + return QueryProcessingStage::Enum::WithMergeableState; + if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) { if (auto projection = getQueryProcessingStageWithAggregateProjection(query_context, storage_snapshot, query_info)) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 512f194ea53..597241c1753 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -876,7 +876,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd ReadFromMergeTree::IndexStats & index_stats, bool use_skip_indexes) { - RangesInDataParts parts_with_ranges(parts.size()); + RangesInDataParts parts_with_ranges; + parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); /// Let's start analyzing all useful indices @@ -1010,7 +1011,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (metadata_snapshot->hasPrimaryKey()) ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings, log); else if (total_marks_count) - ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}}; + ranges.ranges = MarkRanges{{MarkRange{0, total_marks_count}}}; sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp index 0882b7fa129..bd7aa34ec0e 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp @@ -1,4 +1,6 @@ #include +#include "Storages/MergeTree/RangesInDataPart.h" +#include namespace DB { @@ -18,23 +20,42 @@ try initializeReaders(); MarkRanges mark_ranges_for_task; - /// If we need to read few rows, set one range per task to reduce number of read data. - if (has_limit_below_one_block) + + if (!pool) { - mark_ranges_for_task = { std::move(all_mark_ranges.front()) }; - all_mark_ranges.pop_front(); + /// If we need to read few rows, set one range per task to reduce number of read data. + if (has_limit_below_one_block) + { + mark_ranges_for_task = MarkRanges{}; + mark_ranges_for_task.emplace_front(std::move(all_mark_ranges.front())); + all_mark_ranges.pop_front(); + } + else + { + mark_ranges_for_task = std::move(all_mark_ranges); + all_mark_ranges.clear(); + } } else { - mark_ranges_for_task = std::move(all_mark_ranges); - all_mark_ranges.clear(); + auto description = RangesInDataPartDescription{ + .info = data_part->info, + /// We just ignore all the distribution done before + /// Everything will be done on coordinator side + .ranges = {}, + }; + + mark_ranges_for_task = pool->getNewTask(description); + + if (mark_ranges_for_task.empty()) + return false; } auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr : getSizePredictor(data_part, task_columns, sample_block); task = std::make_unique( - data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, task_columns, + data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns, prewhere_info && prewhere_info->remove_prewhere_column, std::move(size_predictor)); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index a6baecee125..84432a293d7 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -2,6 +2,7 @@ #include #include #include +#include "Core/ProtocolDefines.h" namespace DB { @@ -10,6 +11,7 @@ namespace ErrorCodes { extern const int BAD_DATA_PART_NAME; extern const int INVALID_PARTITION_VALUE; + extern const int UNKNOWN_FORMAT_VERSION; } @@ -253,6 +255,43 @@ String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) con return wb.str(); } +void MergeTreePartInfo::serialize(WriteBuffer & out) const +{ + UInt64 version = DBMS_MERGE_TREE_PART_INFO_VERSION; + /// Must be the first + writeIntBinary(version, out); + + writeStringBinary(partition_id, out); + writeIntBinary(min_block, out); + writeIntBinary(max_block, out); + writeIntBinary(level, out); + writeIntBinary(mutation, out); + writeBoolText(use_leagcy_max_level, out); +} + + +String MergeTreePartInfo::describe() const +{ + return getPartNameV1(); +} + + +void MergeTreePartInfo::deserialize(ReadBuffer & in) +{ + UInt64 version; + readIntBinary(version, in); + if (version != DBMS_MERGE_TREE_PART_INFO_VERSION) + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Version for MergeTreePart info mismatched. Got: {}, supported version: {}", + version, DBMS_MERGE_TREE_PART_INFO_VERSION); + + readStringBinary(partition_id, in); + readIntBinary(min_block, in); + readIntBinary(max_block, in); + readIntBinary(level, in); + readIntBinary(mutation, in); + readBoolText(use_leagcy_max_level, in); +} + DetachedPartInfo DetachedPartInfo::parseDetachedPartName( const DiskPtr & disk, std::string_view dir_name, MergeTreeDataFormatVersion format_version) { diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index cad851fb882..b91bc1e595b 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include @@ -112,6 +114,10 @@ struct MergeTreePartInfo return static_cast(max_block - min_block + 1); } + void serialize(WriteBuffer & out) const; + String describe() const; + void deserialize(ReadBuffer & in); + /// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'. static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 37b24422af0..7444e099150 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Common/Stopwatch.h" #include #include @@ -18,6 +19,52 @@ namespace ErrorCodes namespace DB { + +std::vector IMergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & parts) +{ + std::vector per_part_sum_marks; + Block sample_block = storage_snapshot->metadata->getSampleBlock(); + is_part_on_remote_disk.resize(parts.size()); + + for (const auto i : collections::range(0, parts.size())) + { + const auto & part = parts[i]; +#ifndef NDEBUG + assertSortedAndNonIntersecting(part.ranges); +#endif + + bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); + is_part_on_remote_disk[i] = part_on_remote_disk; + do_not_steal_tasks |= part_on_remote_disk; + + /// Read marks for every data part. + size_t sum_marks = 0; + for (const auto & range : part.ranges) + sum_marks += range.end - range.begin; + + per_part_sum_marks.push_back(sum_marks); + + auto task_columns = getReadTaskColumns( + LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, + column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); + + auto size_predictor = !predict_block_size_bytes ? nullptr + : IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block); + + auto & per_part = per_part_params.emplace_back(); + + per_part.data_part = part; + per_part.size_predictor = std::move(size_predictor); + + /// will be used to distinguish between PREWHERE and WHERE columns when applying filter + const auto & required_column_names = task_columns.columns.getNames(); + per_part.column_name_set = {required_column_names.begin(), required_column_names.end()}; + per_part.task_columns = std::move(task_columns); + } + + return per_part_sum_marks; +} + MergeTreeReadPool::MergeTreeReadPool( size_t threads_, size_t sum_marks_, @@ -30,23 +77,25 @@ MergeTreeReadPool::MergeTreeReadPool( const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, bool do_not_steal_tasks_) - : backoff_settings{backoff_settings_} + : IMergeTreeReadPool( + storage_snapshot_, + column_names_, + virtual_column_names_, + min_marks_for_concurrent_read_, + prewhere_info_, + std::move(parts_), + (preferred_block_size_bytes_ > 0), + do_not_steal_tasks_) + , backoff_settings{backoff_settings_} , backoff_state{threads_} - , storage_snapshot{storage_snapshot_} - , column_names{column_names_} - , virtual_column_names{virtual_column_names_} - , do_not_steal_tasks{do_not_steal_tasks_} - , predict_block_size_bytes{preferred_block_size_bytes_ > 0} - , prewhere_info{prewhere_info_} - , parts_ranges{std::move(parts_)} { /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo(parts_ranges); - fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read_); + fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges); } -MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names) +MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread) { const std::lock_guard lock{mutex}; @@ -86,18 +135,18 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t auto & thread_task = thread_tasks.parts_and_ranges.back(); const auto part_idx = thread_task.part_idx; - auto & part = parts_with_idx[part_idx]; + auto & part = per_part_params[part_idx].data_part; auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks need_marks = marks_in_part; else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_to_read); + need_marks = std::min(marks_in_part, min_marks_for_concurrent_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_to_read) + marks_in_part - need_marks < min_marks_for_concurrent_read) need_marks = marks_in_part; MarkRanges ranges_to_get_from_part; @@ -142,7 +191,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t : std::make_unique(*per_part.size_predictor); /// make a copy return std::make_unique( - part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, + part.data_part, ranges_to_get_from_part, part.part_index_in_query, per_part.column_name_set, per_part.task_columns, prewhere_info && prewhere_info->remove_prewhere_column, std::move(curr_task_size_predictor)); } @@ -192,56 +241,9 @@ void MergeTreeReadPool::profileFeedback(ReadBufferFromFileBase::ProfileInfo info } -std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & parts) -{ - std::vector per_part_sum_marks; - Block sample_block = storage_snapshot->metadata->getSampleBlock(); - is_part_on_remote_disk.resize(parts.size()); - - for (const auto i : collections::range(0, parts.size())) - { - const auto & part = parts[i]; -#ifndef NDEBUG - assertSortedAndNonIntersecting(part.ranges); -#endif - - bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); - is_part_on_remote_disk[i] = part_on_remote_disk; - do_not_steal_tasks |= part_on_remote_disk; - - /// Read marks for every data part. - size_t sum_marks = 0; - for (const auto & range : part.ranges) - sum_marks += range.end - range.begin; - - per_part_sum_marks.push_back(sum_marks); - - auto task_columns = getReadTaskColumns( - LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, - column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); - - auto size_predictor = !predict_block_size_bytes ? nullptr - : IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block); - - auto & per_part = per_part_params.emplace_back(); - - per_part.size_predictor = std::move(size_predictor); - - /// will be used to distinguish between PREWHERE and WHERE columns when applying filter - const auto & required_column_names = task_columns.columns.getNames(); - per_part.column_name_set = {required_column_names.begin(), required_column_names.end()}; - per_part.task_columns = std::move(task_columns); - - parts_with_idx.push_back({ part.data_part, part.part_index_in_query }); - } - - return per_part_sum_marks; -} - - void MergeTreeReadPool::fillPerThreadInfo( size_t threads, size_t sum_marks, std::vector per_part_sum_marks, - const RangesInDataParts & parts, size_t min_marks_for_concurrent_read) + const RangesInDataParts & parts) { threads_tasks.resize(threads); if (parts.empty()) @@ -355,4 +357,148 @@ void MergeTreeReadPool::fillPerThreadInfo( } +MergeTreeReadPoolParallelReplicas::~MergeTreeReadPoolParallelReplicas() = default; + + +Block MergeTreeReadPoolParallelReplicas::getHeader() const +{ + return storage_snapshot->getSampleBlockForColumns(extension.colums_to_read); +} + +MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread) +{ + /// This parameter is needed only to satisfy the interface + UNUSED(thread); + + std::lock_guard lock(mutex); + + if (no_more_tasks_available) + return nullptr; + + if (buffered_ranges.empty()) + { + auto result = extension.callback(ParallelReadRequest{ + .replica_num = extension.number_of_current_replica, .min_number_of_marks = min_marks_for_concurrent_read * threads}); + + if (!result || result->finish) + { + no_more_tasks_available = true; + return nullptr; + } + + buffered_ranges = std::move(result->description); + } + + if (buffered_ranges.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No tasks to read. This is a bug"); + + auto & current_task = buffered_ranges.front(); + + RangesInDataPart part; + size_t part_idx = 0; + for (size_t index = 0; index < per_part_params.size(); ++index) + { + auto & other_part = per_part_params[index]; + if (other_part.data_part.data_part->info == current_task.info) + { + part = other_part.data_part; + part_idx = index; + break; + } + } + + MarkRanges ranges_to_read; + size_t current_sum_marks = 0; + while (current_sum_marks < min_marks_for_concurrent_read && !current_task.ranges.empty()) + { + auto diff = min_marks_for_concurrent_read - current_sum_marks; + auto range = current_task.ranges.front(); + if (range.getNumberOfMarks() > diff) + { + auto new_range = range; + new_range.end = range.begin + diff; + range.begin += diff; + + current_task.ranges.front() = range; + ranges_to_read.push_back(new_range); + current_sum_marks += new_range.getNumberOfMarks(); + continue; + } + + ranges_to_read.push_back(range); + current_sum_marks += range.getNumberOfMarks(); + current_task.ranges.pop_front(); + } + + if (current_task.ranges.empty()) + buffered_ranges.pop_front(); + + const auto & per_part = per_part_params[part_idx]; + + auto curr_task_size_predictor + = !per_part.size_predictor ? nullptr : std::make_unique(*per_part.size_predictor); /// make a copy + + return std::make_unique( + part.data_part, + ranges_to_read, + part.part_index_in_query, + per_part.column_name_set, + per_part.task_columns, + prewhere_info && prewhere_info->remove_prewhere_column, + std::move(curr_task_size_predictor)); +} + + +MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPartDescription description) +{ + std::lock_guard lock(mutex); + + auto get_from_buffer = [&]() -> std::optional + { + for (auto & desc : buffered_tasks) + { + if (desc.info == description.info && !desc.ranges.empty()) + { + auto result = std::move(desc.ranges); + desc.ranges = MarkRanges{}; + return result; + } + } + return std::nullopt; + }; + + if (auto result = get_from_buffer(); result) + return result.value(); + + if (no_more_tasks) + return {}; + + auto response = extension.callback(ParallelReadRequest{ + .mode = mode, + .replica_num = extension.number_of_current_replica, + .min_number_of_marks = min_marks_for_concurrent_read * request.size(), + .description = request, + }); + + if (!response || response->description.empty() || response->finish) + { + no_more_tasks = true; + return {}; + } + + /// Fill the buffer + for (size_t i = 0; i < request.size(); ++i) + { + auto & new_ranges = response->description[i].ranges; + auto & old_ranges = buffered_tasks[i].ranges; + std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges)); + } + + if (auto result = get_from_buffer(); result) + return result.value(); + + return {}; +} + + } diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index c9fe70d9a78..46d2e8bae3b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -1,10 +1,13 @@ #pragma once #include -#include +#include #include #include +#include +#include #include + #include @@ -13,6 +16,64 @@ namespace DB using MergeTreeReadTaskPtr = std::unique_ptr; + +class IMergeTreeReadPool +{ +public: + IMergeTreeReadPool( + StorageSnapshotPtr storage_snapshot_, + Names column_names_, + Names virtual_column_names_, + size_t min_marks_for_concurrent_read_, + PrewhereInfoPtr prewhere_info_, + RangesInDataParts parts_ranges_, + bool predict_block_size_bytes_, + bool do_not_steal_tasks_) + : storage_snapshot(storage_snapshot_) + , column_names(column_names_) + , virtual_column_names(virtual_column_names_) + , min_marks_for_concurrent_read(min_marks_for_concurrent_read_) + , prewhere_info(prewhere_info_) + , parts_ranges(parts_ranges_) + , predict_block_size_bytes(predict_block_size_bytes_) + , do_not_steal_tasks(do_not_steal_tasks_) + {} + + virtual MergeTreeReadTaskPtr getTask(size_t thread) = 0; + virtual Block getHeader() const = 0; + virtual void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) = 0; + virtual ~IMergeTreeReadPool() = default; + +protected: + + std::vector fillPerPartInfo(const RangesInDataParts & parts); + + /// Initialized in constructor + StorageSnapshotPtr storage_snapshot; + const Names column_names; + const Names virtual_column_names; + size_t min_marks_for_concurrent_read{0}; + PrewhereInfoPtr prewhere_info; + RangesInDataParts parts_ranges; + bool predict_block_size_bytes; + bool do_not_steal_tasks; + + struct PerPartParams + { + MergeTreeReadTaskColumns task_columns; + NameSet column_name_set; + MergeTreeBlockSizePredictorPtr size_predictor; + RangesInDataPart data_part; + }; + + std::vector per_part_params; + std::vector is_part_on_remote_disk; + + mutable std::mutex mutex; +}; + +using IMergeTreeReadPoolPtr = std::shared_ptr; + /** Provides read tasks for MergeTreeThreadSelectProcessor`s in fine-grained batches, allowing for more * uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads` * workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads @@ -20,7 +81,7 @@ using MergeTreeReadTaskPtr = std::unique_ptr; * it's workload, it either is signaled that no more work is available (`do_not_steal_tasks == false`) or * continues taking small batches from other threads' workloads (`do_not_steal_tasks == true`). */ -class MergeTreeReadPool : private boost::noncopyable +class MergeTreeReadPool final: public IMergeTreeReadPool, private boost::noncopyable { public: /** Pull could dynamically lower (backoff) number of threads, if read operation are too slow. @@ -82,47 +143,22 @@ public: size_t preferred_block_size_bytes_, bool do_not_steal_tasks_ = false); - MergeTreeReadTaskPtr getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names); + ~MergeTreeReadPool() override = default; + MergeTreeReadTaskPtr getTask(size_t thread) override; /** Each worker could call this method and pass information about read performance. * If read performance is too low, pool could decide to lower number of threads: do not assign more tasks to several threads. * This allows to overcome excessive load to disk subsystem, when reads are not from page cache. */ - void profileFeedback(ReadBufferFromFileBase::ProfileInfo info); + void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) override; - Block getHeader() const; + Block getHeader() const override; private: - std::vector fillPerPartInfo(const RangesInDataParts & parts); void fillPerThreadInfo( size_t threads, size_t sum_marks, std::vector per_part_sum_marks, - const RangesInDataParts & parts, size_t min_marks_for_concurrent_read); - - StorageSnapshotPtr storage_snapshot; - const Names column_names; - const Names virtual_column_names; - bool do_not_steal_tasks; - bool predict_block_size_bytes; - - struct PerPartParams - { - MergeTreeReadTaskColumns task_columns; - NameSet column_name_set; - MergeTreeBlockSizePredictorPtr size_predictor; - }; - - std::vector per_part_params; - - PrewhereInfoPtr prewhere_info; - - struct Part - { - MergeTreeData::DataPartPtr data_part; - size_t part_index_in_query; - }; - - std::vector parts_with_idx; + const RangesInDataParts & parts); struct ThreadTask { @@ -137,18 +173,104 @@ private: }; std::vector threads_tasks; - std::set remaining_thread_tasks; - - RangesInDataParts parts_ranges; - - mutable std::mutex mutex; - Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); - std::vector is_part_on_remote_disk; }; using MergeTreeReadPoolPtr = std::shared_ptr; +class MergeTreeReadPoolParallelReplicas : public IMergeTreeReadPool, private boost::noncopyable +{ +public: + + MergeTreeReadPoolParallelReplicas( + StorageSnapshotPtr storage_snapshot_, + size_t threads_, + ParallelReadingExtension extension_, + const RangesInDataParts & parts_, + const PrewhereInfoPtr & prewhere_info_, + const Names & column_names_, + const Names & virtual_column_names_, + size_t min_marks_for_concurrent_read_ + ) + : IMergeTreeReadPool( + storage_snapshot_, + column_names_, + virtual_column_names_, + min_marks_for_concurrent_read_, + prewhere_info_, + parts_, + /*predict_block_size*/false, + /*do_not_steal_tasks*/false) + , extension(extension_) + , threads(threads_) + { + fillPerPartInfo(parts_ranges); + + extension.all_callback({ + .description = parts_ranges.getDescriptions(), + .replica_num = extension.number_of_current_replica + }); + } + + ~MergeTreeReadPoolParallelReplicas() override; + + MergeTreeReadTaskPtr getTask(size_t thread) override; + Block getHeader() const override; + void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {} + +private: + ParallelReadingExtension extension; + + RangesInDataPartsDescription buffered_ranges; + size_t threads; + bool no_more_tasks_available{false}; + Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas"); +}; + +using MergeTreeReadPoolParallelReplicasPtr = std::shared_ptr; + + +class MergeTreeInOrderReadPoolParallelReplicas : private boost::noncopyable +{ +public: + MergeTreeInOrderReadPoolParallelReplicas( + RangesInDataParts parts_, + ParallelReadingExtension extension_, + CoordinationMode mode_, + size_t min_marks_for_concurrent_read_) + : parts_ranges(parts_) + , extension(extension_) + , mode(mode_) + , min_marks_for_concurrent_read(min_marks_for_concurrent_read_) + { + for (const auto & part : parts_ranges) + request.push_back({part.data_part->info, MarkRanges{}}); + + for (const auto & part : parts_ranges) + buffered_tasks.push_back({part.data_part->info, MarkRanges{}}); + + extension.all_callback({ + .description = parts_ranges.getDescriptions(), + .replica_num = extension.number_of_current_replica + }); + } + + MarkRanges getNewTask(RangesInDataPartDescription description); + + RangesInDataParts parts_ranges; + ParallelReadingExtension extension; + CoordinationMode mode; + size_t min_marks_for_concurrent_read{0}; + + bool no_more_tasks{false}; + RangesInDataPartsDescription request; + RangesInDataPartsDescription buffered_tasks; + + std::mutex mutex; +}; + +using MergeTreeInOrderReadPoolParallelReplicasPtr = std::shared_ptr; + } diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index d0d464b3c29..367818c7af1 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -1,4 +1,6 @@ #include +#include +#include "Storages/MergeTree/MergeTreeBaseSelectProcessor.h" namespace DB { @@ -10,6 +12,22 @@ namespace ErrorCodes bool MergeTreeReverseSelectAlgorithm::getNewTaskImpl() try +{ + if (pool) + return getNewTaskParallelReplicas(); + else + return getNewTaskOrdinaryReading(); +} +catch (...) +{ + /// Suspicion of the broken part. A part is added to the queue for verification. + if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + storage.reportBrokenPart(data_part); + throw; +} + + +bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading() { if (chunks.empty() && all_mark_ranges.empty()) return false; @@ -23,25 +41,57 @@ try initializeReaders(); /// Read ranges from right to left. - MarkRanges mark_ranges_for_task = { all_mark_ranges.back() }; + MarkRanges mark_ranges_for_task{std::move(all_mark_ranges.back())}; all_mark_ranges.pop_back(); auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr : getSizePredictor(data_part, task_columns, sample_block); task = std::make_unique( - data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, + data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns, prewhere_info && prewhere_info->remove_prewhere_column, std::move(size_predictor)); return true; + } -catch (...) + +bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas() { - /// Suspicion of the broken part. A part is added to the queue for verification. - if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) - storage.reportBrokenPart(data_part); - throw; + if (chunks.empty() && no_more_tasks) + return false; + + /// We have some blocks to return in buffer. + /// Return true to continue reading, but actually don't create a task. + if (no_more_tasks) + return true; + + if (!reader) + initializeReaders(); + + auto description = RangesInDataPartDescription{ + .info = data_part->info, + /// We just ignore all the distribution done before + /// Everything will be done on coordinator side + .ranges = {}, + }; + + auto mark_ranges_for_task = pool->getNewTask(description); + if (mark_ranges_for_task.empty()) + { + /// If we have chunks in buffer - return true to continue reading from them + return !chunks.empty(); + } + + auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr + : getSizePredictor(data_part, task_columns, sample_block); + + task = std::make_unique( + data_part, mark_ranges_for_task, part_index_in_query, column_name_set, + task_columns, prewhere_info && prewhere_info->remove_prewhere_column, + std::move(size_predictor)); + + return true; } MergeTreeReverseSelectAlgorithm::BlockAndProgress MergeTreeReverseSelectAlgorithm::readFromPart() diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index ccadb1f1c61..fd25748050a 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -27,9 +27,16 @@ private: bool getNewTaskImpl() override; void finalizeNewTask() override {} + bool getNewTaskParallelReplicas(); + bool getNewTaskOrdinaryReading(); + BlockAndProgress readFromPart() override; std::vector chunks; + + /// Used for parallel replicas + bool no_more_tasks{false}; + Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor"); }; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 3f9da9c130a..e5ca851c76b 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -21,21 +21,22 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( const PrewhereInfoPtr & prewhere_info_, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings_, + MergeTreeInOrderReadPoolParallelReplicasPtr pool_, const Names & virt_column_names_, size_t part_index_in_query_, - bool has_limit_below_one_block_, - std::optional extension_) + bool has_limit_below_one_block_) : IMergeTreeSelectAlgorithm{ storage_snapshot_->getSampleBlockForColumns(required_columns_), storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, - reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_}, + reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, data_part{owned_data_part_}, sample_block(storage_snapshot_->metadata->getSampleBlock()), all_mark_ranges(std::move(mark_ranges_)), part_index_in_query(part_index_in_query_), has_limit_below_one_block(has_limit_below_one_block_), + pool(pool_), total_rows(data_part->index_granularity.getRowsCountInRanges(all_mark_ranges)) { ordered_names = header_without_const_virtual_columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 12f4804835c..76c8d81dd0b 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -29,10 +30,10 @@ public: const PrewhereInfoPtr & prewhere_info, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings, + MergeTreeInOrderReadPoolParallelReplicasPtr pool_, const Names & virt_column_names = {}, size_t part_index_in_query_ = 0, - bool has_limit_below_one_block_ = false, - std::optional extension_ = {}); + bool has_limit_below_one_block_ = false); ~MergeTreeSelectAlgorithm() override; @@ -64,6 +65,9 @@ protected: /// It reduces amount of read data for queries with small LIMIT. bool has_limit_below_one_block = false; + /// Pool for reading in order + MergeTreeInOrderReadPoolParallelReplicasPtr pool; + size_t total_rows = 0; }; diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 60586024359..97c283b8c01 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -7,16 +7,11 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm( size_t thread_, - const MergeTreeReadPoolPtr & pool_, - size_t min_marks_to_read_, - UInt64 max_block_size_rows_, + IMergeTreeReadPoolPtr pool_, + size_t min_marks_for_concurrent_read_, + size_t max_block_size_rows_, size_t preferred_block_size_bytes_, size_t preferred_max_column_in_block_size_bytes_, const MergeTreeData & storage_, @@ -25,74 +20,22 @@ MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm( const PrewhereInfoPtr & prewhere_info_, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings_, - const Names & virt_column_names_, - std::optional extension_) + const Names & virt_column_names_) : IMergeTreeSelectAlgorithm{ pool_->getHeader(), storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, - reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_}, + reader_settings_, use_uncompressed_cache_, virt_column_names_}, thread{thread_}, - pool{pool_} + pool{std::move(pool_)} { - /// round min_marks_to_read up to nearest multiple of block_size expressed in marks - /// If granularity is adaptive it doesn't make sense - /// Maybe it will make sense to add settings `max_block_size_bytes` - if (max_block_size_rows && !storage.canUseAdaptiveGranularity()) - { - size_t fixed_index_granularity = storage.getSettings()->index_granularity; - min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1) - / max_block_size_rows * max_block_size_rows / fixed_index_granularity; - } - else if (extension.has_value()) - { - /// Parallel reading from replicas is enabled. - /// We try to estimate the average number of bytes in a granule - /// to make one request over the network per one gigabyte of data - /// Actually we will ask MergeTreeReadPool to provide us heavier tasks to read - /// because the most part of each task will be postponed - /// (due to using consistent hash for better cache affinity) - const size_t amount_of_read_bytes_per_one_request = 1024 * 1024 * 1024; // 1GiB - /// In case of reading from compact parts (for which we can't estimate the average size of marks) - /// we will use this value - const size_t empirical_size_of_mark = 1024 * 1024 * 10; // 10 MiB - - if (extension->colums_to_read.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "A set of column to read is empty. It is a bug"); - - size_t sum_average_marks_size = 0; - auto column_sizes = storage.getColumnSizes(); - for (const auto & name : extension->colums_to_read) - { - auto it = column_sizes.find(name); - if (it == column_sizes.end()) - continue; - auto size = it->second; - - if (size.data_compressed == 0 || size.data_uncompressed == 0 || size.marks == 0) - continue; - - sum_average_marks_size += size.data_uncompressed / size.marks; - } - - if (sum_average_marks_size == 0) - sum_average_marks_size = empirical_size_of_mark * extension->colums_to_read.size(); - - min_marks_to_read = extension->count_participating_replicas * amount_of_read_bytes_per_one_request / sum_average_marks_size; - } - else - { - min_marks_to_read = min_marks_to_read_; - } - - - ordered_names = getHeader().getNames(); + min_marks_to_read = min_marks_for_concurrent_read_; } /// Requests read task from MergeTreeReadPool and signals whether it got one bool MergeTreeThreadSelectAlgorithm::getNewTaskImpl() { - task = pool->getTask(min_marks_to_read, thread, ordered_names); + task = pool->getTask(thread); return static_cast(task); } @@ -113,19 +56,19 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask() owned_uncompressed_cache = storage.getContext()->getUncompressedCache(); owned_mark_cache = storage.getContext()->getMarkCache(); } - else if (part_name != last_readed_part_name) + else if (part_name != last_read_part_name) { value_size_map = reader->getAvgValueSizeHints(); } - const bool init_new_readers = !reader || part_name != last_readed_part_name; + const bool init_new_readers = !reader || part_name != last_read_part_name; if (init_new_readers) { initializeMergeTreeReadersForPart(task->data_part, task->task_columns, metadata_snapshot, task->mark_ranges, value_size_map, profile_callback); } - last_readed_part_name = part_name; + last_read_part_name = part_name; } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h index ac3dcf0cc41..37c9375a581 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h @@ -5,8 +5,8 @@ namespace DB { -class MergeTreeReadPool; - +class IMergeTreeReadPool; +using IMergeTreeReadPoolPtr = std::shared_ptr; /** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked * to perform. @@ -16,9 +16,9 @@ class MergeTreeThreadSelectAlgorithm final : public IMergeTreeSelectAlgorithm public: MergeTreeThreadSelectAlgorithm( size_t thread_, - const std::shared_ptr & pool_, - size_t min_marks_to_read_, - UInt64 max_block_size_, + IMergeTreeReadPoolPtr pool_, + size_t min_marks_for_concurrent_read, + size_t max_block_size_, size_t preferred_block_size_bytes_, size_t preferred_max_column_in_block_size_bytes_, const MergeTreeData & storage_, @@ -27,8 +27,7 @@ public: const PrewhereInfoPtr & prewhere_info_, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings_, - const Names & virt_column_names_, - std::optional extension_); + const Names & virt_column_names_); String getName() const override { return "MergeTreeThread"; } @@ -42,18 +41,14 @@ protected: void finish() override; - bool canUseConsistentHashingForParallelReading() override { return true; } - private: /// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1]) size_t thread; - std::shared_ptr pool; + IMergeTreeReadPoolPtr pool; /// Last part read in this thread - std::string last_readed_part_name; - /// Names from header. Used in order to order columns in read blocks. - Names ordered_names; + std::string last_read_part_name; }; } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index e07f19fb64c..3ef064ff743 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1,23 +1,95 @@ #include +#include +#include +#include +#include #include +#include + +#include #include -#include -#include -#include +#include +#include +#include +#include "IO/WriteBufferFromString.h" +#include "Storages/MergeTree/RangesInDataPart.h" +#include "Storages/MergeTree/RequestResponse.h" +#include #include +#include namespace DB { -class ParallelReplicasReadingCoordinator::Impl +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class ParallelReplicasReadingCoordinator::ImplInterface { public: - using PartitionReadRequestPtr = std::unique_ptr; + struct Stat + { + size_t number_of_requests{0}; + size_t sum_marks{0}; + }; + using Stats = std::vector; + static String toString(Stats stats) + { + String result = "Statistics: "; + for (size_t i = 0; i < stats.size(); ++i) + result += fmt::format("-- replica {}, requests: {} marks: {} ", i, stats[i].number_of_requests, stats[i].sum_marks); + return result; + } + + Stats stats; + std::mutex mutex; + size_t replicas_count; + + explicit ImplInterface(size_t replicas_count_) + : stats{replicas_count_} + , replicas_count(replicas_count_) + {} + + virtual ~ImplInterface() = default; + virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0; + virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0; +}; + + +struct Part +{ + mutable RangesInDataPartDescription description; + // FIXME: This is needed to put this struct in set + // and modify through iterator + mutable std::set replicas; + + bool operator<(const Part & rhs) const { return description.info < rhs.description.info; } +}; + +using Parts = std::set; +using PartRefs = std::deque; + + +class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface +{ +public: + using ParallelReadRequestPtr = std::unique_ptr; using PartToMarkRanges = std::map; + explicit DefaultCoordinator(size_t replicas_count_) + : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) + , announcements(replicas_count_) + , reading_state(replicas_count_) + { + } + + ~DefaultCoordinator() override; + struct PartitionReading { PartSegments part_ranges; @@ -27,115 +99,423 @@ public: using PartitionToBlockRanges = std::map; PartitionToBlockRanges partitions; - std::mutex mutex; + size_t sent_initial_requests{0}; + std::vector announcements; - PartitionReadResponse handleRequest(PartitionReadRequest request); + Parts all_parts_to_read; + /// Contains only parts which we haven't started to read from + PartRefs delayed_parts; + /// Per-replica preferred parts split by consistent hash + /// Once all task will be done by some replica, it can steal tasks + std::vector reading_state; + + Poco::Logger * log = &Poco::Logger::get("DefaultCoordinator"); + + std::atomic state_initialized{false}; + + ParallelReadResponse handleRequest(ParallelReadRequest request) override; + void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; + + void updateReadingState(const InitialAllRangesAnnouncement & announcement); + void finalizeReadingState(); + + size_t computeConsistentHash(const MergeTreePartInfo & info) const + { + auto hash = SipHash(); + hash.update(info.getPartNameV1()); + return ConsistentHashing(hash.get64(), replicas_count); + } + + void selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const; +}; + +DefaultCoordinator::~DefaultCoordinator() +{ + LOG_INFO(log, "Coordination done: {}", toString(stats)); +} + +void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & announcement) +{ + PartRefs parts_diff; + + /// To get rid of duplicates + for (const auto & part: announcement.description) + { + auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), + [&part] (const Part & other) { return other.description.info.getPartNameV1() == part.info.getPartNameV1(); }); + + /// We have the same part - add the info about presence on current replica to it + if (the_same_it != all_parts_to_read.end()) + { + the_same_it->replicas.insert(announcement.replica_num); + continue; + } + + auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), + [&part] (const Part & other) { return !other.description.info.isDisjoint(part.info); }); + + /// It is covering part or we have covering - skip it + if (covering_or_the_same_it != all_parts_to_read.end()) + continue; + + auto new_part = Part{ + .description = part, + .replicas = {announcement.replica_num} + }; + + auto [insert_it, _] = all_parts_to_read.insert(new_part); + parts_diff.push_back(insert_it); + } + + /// Split all parts by consistent hash + while (!parts_diff.empty()) + { + auto current_part_it = parts_diff.front(); + parts_diff.pop_front(); + auto consistent_hash = computeConsistentHash(current_part_it->description.info); + + /// Check whether the new part can easy go to replica queue + if (current_part_it->replicas.contains(consistent_hash)) + { + reading_state[consistent_hash].emplace_back(current_part_it); + continue; + } + + /// Add to delayed parts + delayed_parts.emplace_back(current_part_it); + } +} + +void DefaultCoordinator::finalizeReadingState() +{ + /// Clear all the delayed queue + while (!delayed_parts.empty()) + { + auto current_part_it = delayed_parts.front(); + auto consistent_hash = computeConsistentHash(current_part_it->description.info); + + if (current_part_it->replicas.contains(consistent_hash)) + { + reading_state[consistent_hash].emplace_back(current_part_it); + delayed_parts.pop_front(); + continue; + } + + /// In this situation just assign to a random replica which has this part + auto replica = *(std::next(current_part_it->replicas.begin(), thread_local_rng() % current_part_it->replicas.size())); + reading_state[replica].emplace_back(current_part_it); + delayed_parts.pop_front(); + } + + String description; + for (const auto & part : all_parts_to_read) + { + description += part.description.describe(); + description += fmt::format("Replicas: ({}) --- ", fmt::join(part.replicas, ",")); + } + + LOG_INFO(log, "Reading state is fully initialized: {}", description); +} + + +void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) +{ + std::lock_guard lock(mutex); + + updateReadingState(announcement); + stats[announcement.replica_num].number_of_requests +=1; + + ++sent_initial_requests; + LOG_INFO(log, "{} {}", sent_initial_requests, replicas_count); + if (sent_initial_requests == replicas_count) + finalizeReadingState(); +} + +void DefaultCoordinator::selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const +{ + for (const auto & part : container) + { + if (current_mark_size >= min_number_of_marks) + { + LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_mark_size, min_number_of_marks); + break; + } + + if (part->description.ranges.empty()) + { + LOG_TEST(log, "Part {} is already empty in reading state", part->description.info.getPartNameV1()); + continue; + } + + if (std::find(part->replicas.begin(), part->replicas.end(), replica_num) == part->replicas.end()) + { + LOG_TEST(log, "Not found part {} on replica {}", part->description.info.getPartNameV1(), replica_num); + continue; + } + + response.description.push_back({ + .info = part->description.info, + .ranges = {}, + }); + + while (!part->description.ranges.empty() && current_mark_size < min_number_of_marks) + { + auto & range = part->description.ranges.front(); + + if (range.getNumberOfMarks() > min_number_of_marks) + { + auto new_range = range; + range.begin += min_number_of_marks; + new_range.end = new_range.begin + min_number_of_marks; + + response.description.back().ranges.emplace_back(new_range); + current_mark_size += new_range.getNumberOfMarks(); + continue; + } + + current_mark_size += part->description.ranges.front().getNumberOfMarks(); + response.description.back().ranges.emplace_back(part->description.ranges.front()); + part->description.ranges.pop_front(); + } + } +} + +ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest request) +{ + std::lock_guard lock(mutex); + + LOG_TRACE(log, "Handling request from replica {}, minimal marks size is {}", request.replica_num, request.min_number_of_marks); + + size_t current_mark_size = 0; + ParallelReadResponse response; + + /// 1. Try to select from preferred set of parts for current replica + selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response); + + /// 2. Try to use parts from delayed queue + while (!delayed_parts.empty() && current_mark_size < request.min_number_of_marks) + { + auto part = delayed_parts.front(); + delayed_parts.pop_front(); + reading_state[request.replica_num].emplace_back(part); + selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response); + } + + /// 3. Try to steal tasks; + if (current_mark_size < request.min_number_of_marks) + { + for (size_t i = 0; i < replicas_count; ++i) + { + if (i != request.replica_num) + selectPartsAndRanges(reading_state[i], request.replica_num, request.min_number_of_marks, current_mark_size, response); + + if (current_mark_size >= request.min_number_of_marks) + break; + } + } + + stats[request.replica_num].number_of_requests += 1; + stats[request.replica_num].sum_marks += current_mark_size; + + if (response.description.empty()) + response.finish = true; + + LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe()); + return response; +} + + +template +class InOrderCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface +{ +public: + explicit InOrderCoordinator([[ maybe_unused ]] size_t replicas_count_) + : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) + {} + ~InOrderCoordinator() override + { + LOG_INFO(log, "Coordination done: {}", toString(stats)); + } + + ParallelReadResponse handleRequest([[ maybe_unused ]] ParallelReadRequest request) override; + void handleInitialAllRangesAnnouncement([[ maybe_unused ]] InitialAllRangesAnnouncement announcement) override; + + Parts all_parts_to_read; + + Poco::Logger * log = &Poco::Logger::get(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; -PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(PartitionReadRequest request) +template +void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator"); - Stopwatch watch; - - String request_description = request.toString(); std::lock_guard lock(mutex); + LOG_TRACE(log, "Received an announecement {}", announcement.describe()); - auto partition_it = partitions.find(request.partition_id); - - PartToRead::PartAndProjectionNames part_and_projection + /// To get rid of duplicates + for (const auto & part: announcement.description) { - .part = request.part_name, - .projection = request.projection_name - }; + auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), + [&part] (const Part & other) { return other.description.info == part.info; }); - /// We are the first who wants to process parts in partition - if (partition_it == partitions.end()) - { - PartitionReading partition_reading; + /// We have the same part - add the info about presence on current replica to it + if (the_same_it != all_parts_to_read.end()) + { + the_same_it->replicas.insert(announcement.replica_num); + continue; + } - PartToRead part_to_read; - part_to_read.range = request.block_range; - part_to_read.name = part_and_projection; + auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), + [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); - partition_reading.part_ranges.addPart(std::move(part_to_read)); + /// It is covering part or we have covering - skip it + if (covering_or_the_same_it != all_parts_to_read.end()) + continue; - /// As this query is first in partition, we will accept all ranges from it. - /// We need just to update our state. - auto request_ranges = HalfIntervals::initializeFromMarkRanges(request.mark_ranges); - auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace(); - mark_ranges_index.intersect(request_ranges.negate()); + auto new_part = Part{ + .description = part, + .replicas = {announcement.replica_num} + }; - partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); - partitions.insert({request.partition_id, std::move(partition_reading)}); - - LOG_TRACE(log, "Request is first in partition, accepted in {} ns: {}", watch.elapsed(), request_description); - return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; + auto insert_it = all_parts_to_read.insert(new_part); + auto & ranges = insert_it.first->description.ranges; + std::sort(ranges.begin(), ranges.end()); } - - auto & partition_reading = partition_it->second; - - PartToRead part_to_read; - part_to_read.range = request.block_range; - part_to_read.name = part_and_projection; - - auto part_intersection_res = partition_reading.part_ranges.getIntersectionResult(part_to_read); - - switch (part_intersection_res) - { - case PartSegments::IntersectionResult::REJECT: - { - LOG_TRACE(log, "Request rejected in {} ns: {}", watch.elapsed(), request_description); - return {.denied = true, .mark_ranges = {}}; - } - case PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION: - { - auto marks_it = partition_reading.mark_ranges_in_part.find(part_and_projection); - - auto & intervals_to_do = marks_it->second; - auto result = HalfIntervals::initializeFromMarkRanges(request.mark_ranges); - result.intersect(intervals_to_do); - - /// Update intervals_to_do - intervals_to_do.intersect(HalfIntervals::initializeFromMarkRanges(std::move(request.mark_ranges)).negate()); - - auto result_ranges = result.convertToMarkRangesFinal(); - const bool denied = result_ranges.empty(); - - if (denied) - LOG_TRACE(log, "Request rejected due to intersection in {} ns: {}", watch.elapsed(), request_description); - else - LOG_TRACE(log, "Request accepted partially in {} ns: {}", watch.elapsed(), request_description); - - return {.denied = denied, .mark_ranges = std::move(result_ranges)}; - } - case PartSegments::IntersectionResult::NO_INTERSECTION: - { - partition_reading.part_ranges.addPart(std::move(part_to_read)); - - auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace().intersect( - HalfIntervals::initializeFromMarkRanges(request.mark_ranges).negate() - ); - partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); - - LOG_TRACE(log, "Request accepted in {} ns: {}", watch.elapsed(), request_description); - return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; - } - } - - UNREACHABLE(); } -PartitionReadResponse ParallelReplicasReadingCoordinator::handleRequest(PartitionReadRequest request) + +template +ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest request) { + std::lock_guard lock(mutex); + + if (request.mode != mode) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Replica {} decided to read in {} mode, not in {}. This is a bug", + request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode)); + + LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe()); + + ParallelReadResponse response; + response.description = request.description; + size_t overall_number_of_marks = 0; + + for (auto & part : response.description) + { + auto global_part_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), + [&part] (const Part & other) { return other.description.info == part.info; }); + + if (global_part_it == all_parts_to_read.end()) + continue; + + if (!global_part_it->replicas.contains(request.replica_num)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num); + + size_t current_mark_size = 0; + + /// Now we can recommend to read more intervals + if constexpr (mode == CoordinationMode::ReverseOrder) + { + while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks) + { + auto range = global_part_it->description.ranges.back(); + + if (range.getNumberOfMarks() > request.min_number_of_marks) + { + auto new_range = range; + range.end -= request.min_number_of_marks; + new_range.begin = new_range.end - request.min_number_of_marks; + + global_part_it->description.ranges.back() = range; + + part.ranges.emplace_front(new_range); + current_mark_size += new_range.getNumberOfMarks(); + continue; + } + + current_mark_size += global_part_it->description.ranges.back().getNumberOfMarks(); + part.ranges.emplace_front(global_part_it->description.ranges.back()); + global_part_it->description.ranges.pop_back(); + } + } + else if constexpr (mode == CoordinationMode::WithOrder) + { + while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks) + { + auto range = global_part_it->description.ranges.front(); + + if (range.getNumberOfMarks() > request.min_number_of_marks) + { + auto new_range = range; + range.begin += request.min_number_of_marks; + new_range.end = new_range.begin + request.min_number_of_marks; + + global_part_it->description.ranges.front() = range; + + part.ranges.emplace_back(new_range); + current_mark_size += new_range.getNumberOfMarks(); + continue; + } + + current_mark_size += global_part_it->description.ranges.front().getNumberOfMarks(); + part.ranges.emplace_back(global_part_it->description.ranges.front()); + global_part_it->description.ranges.pop_front(); + } + } + + overall_number_of_marks += current_mark_size; + } + + if (!overall_number_of_marks) + response.finish = true; + + stats[request.replica_num].number_of_requests += 1; + stats[request.replica_num].sum_marks += overall_number_of_marks; + + LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe()); + return response; +} + + +void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) +{ + if (!pimpl) + initialize(); + + return pimpl->handleInitialAllRangesAnnouncement(announcement); +} + +ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request) +{ + if (!pimpl) + initialize(); + return pimpl->handleRequest(std::move(request)); } -ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator() +void ParallelReplicasReadingCoordinator::setMode(CoordinationMode mode_) { - pimpl = std::make_unique(); + mode = mode_; } +void ParallelReplicasReadingCoordinator::initialize() +{ + switch (mode) + { + case CoordinationMode::Default: + pimpl = std::make_unique(replicas_count); + return; + case CoordinationMode::WithOrder: + pimpl = std::make_unique>(replicas_count); + return; + case CoordinationMode::ReverseOrder: + pimpl = std::make_unique>(replicas_count); + return; + } +} + +ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) {} + ParallelReplicasReadingCoordinator::~ParallelReplicasReadingCoordinator() = default; } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 4800533e919..0656a128884 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -7,15 +7,28 @@ namespace DB { +/// The main class to spread mark ranges across replicas dynamically +/// The reason why it uses pimpl - this header file is included in +/// multiple other files like Context or RemoteQueryExecutor class ParallelReplicasReadingCoordinator { public: - ParallelReplicasReadingCoordinator(); + class ImplInterface; + + explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); ~ParallelReplicasReadingCoordinator(); - PartitionReadResponse handleRequest(PartitionReadRequest request); + + void setMode(CoordinationMode mode); + void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); + ParallelReadResponse handleRequest(ParallelReadRequest request); + private: - class Impl; - std::unique_ptr pimpl; + void initialize(); + + CoordinationMode mode{CoordinationMode::Default}; + size_t replicas_count{0}; + std::atomic initialized{false}; + std::unique_ptr pimpl; }; using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/RangesInDataPart.cpp b/src/Storages/MergeTree/RangesInDataPart.cpp new file mode 100644 index 00000000000..29a236c9865 --- /dev/null +++ b/src/Storages/MergeTree/RangesInDataPart.cpp @@ -0,0 +1,113 @@ +#include + +#include + +#include "IO/VarInt.h" + +#include +#include + + +namespace DB +{ + +void RangesInDataPartDescription::serialize(WriteBuffer & out) const +{ + info.serialize(out); + ranges.serialize(out); +} + +String RangesInDataPartDescription::describe() const +{ + String result; + result += fmt::format("Part: {}, ", info.getPartNameV1()); + result += fmt::format("Ranges: [{}], ", fmt::join(ranges, ",")); + return result; +} + +void RangesInDataPartDescription::deserialize(ReadBuffer & in) +{ + info.deserialize(in); + ranges.deserialize(in); +} + +void RangesInDataPartsDescription::serialize(WriteBuffer & out) const +{ + writeVarUInt(this->size(), out); + for (const auto & desc : *this) + desc.serialize(out); +} + +String RangesInDataPartsDescription::describe() const +{ + String result; + for (const auto & desc : *this) + result += desc.describe() + ","; + return result; +} + +void RangesInDataPartsDescription::deserialize(ReadBuffer & in) +{ + size_t new_size = 0; + readVarUInt(new_size, in); + + this->resize(new_size); + for (auto & desc : *this) + desc.deserialize(in); +} + +void RangesInDataPartsDescription::merge(RangesInDataPartsDescription & other) +{ + for (const auto & desc : other) + this->emplace_back(desc); +} + +RangesInDataPartDescription RangesInDataPart::getDescription() const +{ + return RangesInDataPartDescription{ + .info = data_part->info, + .ranges = ranges, + }; +} + +size_t RangesInDataPart::getMarksCount() const +{ + size_t total = 0; + for (const auto & range : ranges) + total += range.end - range.begin; + + return total; +} + +size_t RangesInDataPart::getRowsCount() const +{ + return data_part->index_granularity.getRowsCountInRanges(ranges); +} + + +RangesInDataPartsDescription RangesInDataParts::getDescriptions() const +{ + RangesInDataPartsDescription result; + for (const auto & part : *this) + result.emplace_back(part.getDescription()); + return result; +} + + +size_t RangesInDataParts::getMarksCountAllParts() const +{ + size_t result = 0; + for (const auto & part : *this) + result += part.getMarksCount(); + return result; +} + +size_t RangesInDataParts::getRowsCountAllParts() const +{ + size_t result = 0; + for (const auto & part: *this) + result += part.getRowsCount(); + return result; +} + +} diff --git a/src/Storages/MergeTree/RangesInDataPart.h b/src/Storages/MergeTree/RangesInDataPart.h index 4f5d34e118d..9c8ab4859a0 100644 --- a/src/Storages/MergeTree/RangesInDataPart.h +++ b/src/Storages/MergeTree/RangesInDataPart.h @@ -1,42 +1,73 @@ #pragma once -#include +#include + +#include +#include #include +#include "Storages/MergeTree/MergeTreePartInfo.h" namespace DB { +class IMergeTreeDataPart; +using DataPartPtr = std::shared_ptr; + +/// The only purpose of this struct is that serialize and deserialize methods +/// they look natural here because we can fully serialize and then deserialize original DataPart class. +struct RangesInDataPartDescription +{ + MergeTreePartInfo info; + MarkRanges ranges; + + void serialize(WriteBuffer & out) const; + String describe() const; + void deserialize(ReadBuffer & in); +}; + +struct RangesInDataPartsDescription: public std::deque +{ + using std::deque::deque; + + void serialize(WriteBuffer & out) const; + String describe() const; + void deserialize(ReadBuffer & in); + + void merge(RangesInDataPartsDescription & other); +}; struct RangesInDataPart { - MergeTreeData::DataPartPtr data_part; + DataPartPtr data_part; size_t part_index_in_query; MarkRanges ranges; RangesInDataPart() = default; - RangesInDataPart(const MergeTreeData::DataPartPtr & data_part_, const size_t part_index_in_query_, - const MarkRanges & ranges_ = MarkRanges{}) - : data_part{data_part_}, part_index_in_query{part_index_in_query_}, ranges{ranges_} - { - } + RangesInDataPart( + const DataPartPtr & data_part_, + const size_t part_index_in_query_, + const MarkRanges & ranges_ = MarkRanges{}) + : data_part{data_part_} + , part_index_in_query{part_index_in_query_} + , ranges{ranges_} + {} - size_t getMarksCount() const - { - size_t total = 0; - for (const auto & range : ranges) - total += range.end - range.begin; + RangesInDataPartDescription getDescription() const; - return total; - } - - size_t getRowsCount() const - { - return data_part->index_granularity.getRowsCountInRanges(ranges); - } + size_t getMarksCount() const; + size_t getRowsCount() const; }; -using RangesInDataParts = std::vector; +struct RangesInDataParts: public std::vector +{ + using std::vector::vector; + + RangesInDataPartsDescription getDescriptions() const; + + size_t getMarksCountAllParts() const; + size_t getRowsCountAllParts() const; +}; } diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 2ea6b0c9f9f..5249128590f 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -1,159 +1,129 @@ +#include #include #include #include +#include "IO/VarInt.h" #include #include -#include #include - namespace DB { namespace ErrorCodes { extern const int UNKNOWN_PROTOCOL; - extern const int BAD_ARGUMENTS; } -static void readMarkRangesBinary(MarkRanges & ranges, ReadBuffer & buf) -{ - size_t size = 0; - readVarUInt(size, buf); - - if (size > DEFAULT_MAX_STRING_SIZE) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Too large ranges size: {}.", size); - - ranges.resize(size); - for (size_t i = 0; i < size; ++i) - { - readBinary(ranges[i].begin, buf); - readBinary(ranges[i].end, buf); - } -} - - -static void writeMarkRangesBinary(const MarkRanges & ranges, WriteBuffer & buf) -{ - writeVarUInt(ranges.size(), buf); - - for (const auto & [begin, end] : ranges) - { - writeBinary(begin, buf); - writeBinary(end, buf); - } -} - - -void PartitionReadRequest::serialize(WriteBuffer & out) const +void ParallelReadRequest::serialize(WriteBuffer & out) const { + UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + writeIntBinary(version, out); - writeStringBinary(partition_id, out); - writeStringBinary(part_name, out); - writeStringBinary(projection_name, out); - - writeVarInt(block_range.begin, out); - writeVarInt(block_range.end, out); - - writeMarkRangesBinary(mark_ranges, out); + writeIntBinary(mode, out); + writeIntBinary(replica_num, out); + writeIntBinary(min_number_of_marks, out); + description.serialize(out); } -String PartitionReadRequest::toString() const +String ParallelReadRequest::describe() const { - WriteBufferFromOwnString out; - out << "partition: " << partition_id << ", part: " << part_name; - if (!projection_name.empty()) - out << ", projection: " << projection_name; - out << ", block range: [" << block_range.begin << ", " << block_range.end << "]"; - out << ", mark ranges: "; - - bool is_first = true; - for (const auto & [begin, end] : mark_ranges) - { - if (!is_first) - out << ", "; - out << "[" << begin << ", " << end << ")"; - is_first = false; - } - - return out.str(); + String result; + result += fmt::format("replica_num: {} \n", replica_num); + result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks); + result += description.describe(); + return result; } - -void PartitionReadRequest::deserialize(ReadBuffer & in) +void ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; - readVarUInt(version, in); + readIntBinary(version, in); if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \ - from replicas differ. Got: {}, supported version: {}", + throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading "\ + "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); - readStringBinary(partition_id, in); - readStringBinary(part_name, in); - readStringBinary(projection_name, in); - - readVarInt(block_range.begin, in); - readVarInt(block_range.end, in); - - readMarkRangesBinary(mark_ranges, in); + readIntBinary(mode, in); + readIntBinary(replica_num, in); + readIntBinary(min_number_of_marks, in); + description.deserialize(in); } -UInt64 PartitionReadRequest::getConsistentHash(size_t buckets) const +void ParallelReadRequest::merge(ParallelReadRequest & other) { - SipHash hash; - - hash.update(partition_id.size()); - hash.update(partition_id); - - hash.update(part_name.size()); - hash.update(part_name); - - hash.update(projection_name.size()); - hash.update(projection_name); - - hash.update(block_range.begin); - hash.update(block_range.end); - - hash.update(mark_ranges.size()); - for (const auto & range : mark_ranges) - { - hash.update(range.begin); - hash.update(range.end); - } - - return ConsistentHashing(hash.get64(), buckets); + assert(mode == other.mode); + assert(replica_num == other.replica_num); + assert(min_number_of_marks == other.min_number_of_marks); + description.merge(other.description); } - -void PartitionReadResponse::serialize(WriteBuffer & out) const +void ParallelReadResponse::serialize(WriteBuffer & out) const { + UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + writeIntBinary(version, out); - writeBinary(denied, out); - writeMarkRangesBinary(mark_ranges, out); + writeBoolText(finish, out); + description.serialize(out); } +String ParallelReadResponse::describe() const +{ + String result; + result += fmt::format("finish: {} \n", finish); + result += description.describe(); + return result; +} -void PartitionReadResponse::deserialize(ReadBuffer & in) +void ParallelReadResponse::deserialize(ReadBuffer & in) { UInt64 version; - readVarUInt(version, in); + readIntBinary(version, in); if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \ - from replicas differ. Got: {}, supported version: {}", + throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ + "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); - UInt64 value; - readVarUInt(value, in); - denied = static_cast(value); - readMarkRangesBinary(mark_ranges, in); + readBoolText(finish, in); + description.deserialize(in); +} + + +void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const +{ + UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; + /// Must be the first + writeIntBinary(version, out); + + description.serialize(out); + writeIntBinary(replica_num, out); +} + + +String InitialAllRangesAnnouncement::describe() +{ + String result; + result += description.describe(); + result += fmt::format("----------\nReceived from {} replica\n", replica_num); + return result; +} + +void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) +{ + UInt64 version; + readIntBinary(version, in); + if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ + "from replicas differ. Got: {}, supported version: {}", + version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + + description.deserialize(in); + readIntBinary(replica_num, in); } } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index ce9dc55f479..8cdb9e49be5 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -9,12 +10,21 @@ #include #include +#include namespace DB { -/// Represents a segment [left; right] of part's block numbers. +enum class CoordinationMode +{ + Default, + /// For reading in order + WithOrder, + ReverseOrder +}; + +/// Represents a segment [left; right] struct PartBlockRange { Int64 begin; @@ -26,34 +36,44 @@ struct PartBlockRange } }; -struct PartitionReadRequest +struct ParallelReadRequest { - String partition_id; - String part_name; - String projection_name; - PartBlockRange block_range; - MarkRanges mark_ranges; + CoordinationMode mode; + size_t replica_num; + size_t min_number_of_marks; + + /// Extension for ordered mode + RangesInDataPartsDescription description; void serialize(WriteBuffer & out) const; + String describe() const; void deserialize(ReadBuffer & in); - - UInt64 getConsistentHash(size_t buckets) const; - - /// Describe it for debugging purposes. - String toString() const; + void merge(ParallelReadRequest & other); }; -struct PartitionReadResponse +struct ParallelReadResponse { - bool denied{false}; - MarkRanges mark_ranges{}; + bool finish{false}; + RangesInDataPartsDescription description; void serialize(WriteBuffer & out) const; + String describe() const; void deserialize(ReadBuffer & in); }; -using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; +struct InitialAllRangesAnnouncement +{ + RangesInDataPartsDescription description; + size_t replica_num; + void serialize(WriteBuffer & out) const; + String describe(); + void deserialize(ReadBuffer & in); +}; + + +using MergeTreeAllRangesCallback = std::function; +using MergeTreeReadTaskCallback = std::function(ParallelReadRequest)>; } diff --git a/src/Storages/MergeTree/tests/gtest_coordinator.cpp b/src/Storages/MergeTree/tests/gtest_coordinator.cpp deleted file mode 100644 index 7bcf3304c2b..00000000000 --- a/src/Storages/MergeTree/tests/gtest_coordinator.cpp +++ /dev/null @@ -1,240 +0,0 @@ -#include - -#include -#include -#include - -#include - -#include - -using namespace DB; - - -TEST(HalfIntervals, Simple) -{ - ASSERT_TRUE(( - HalfIntervals{{{1, 2}, {3, 4}}}.negate() == - HalfIntervals{{{0, 1}, {2, 3}, {4, 18446744073709551615UL}}} - )); - - { - auto left = HalfIntervals{{{0, 2}, {4, 6}}}.negate(); - ASSERT_TRUE(( - left == - HalfIntervals{{{2, 4}, {6, 18446744073709551615UL}}} - )); - } - - { - auto left = HalfIntervals{{{0, 2}, {4, 6}}}; - auto right = HalfIntervals{{{1, 5}}}.negate(); - auto intersection = left.intersect(right); - - ASSERT_TRUE(( - intersection == - HalfIntervals{{{0, 1}, {5, 6}}} - )); - } - - { - auto left = HalfIntervals{{{1, 2}, {2, 3}}}; - auto right = HalfIntervals::initializeWithEntireSpace(); - auto intersection = right.intersect(left.negate()); - - ASSERT_TRUE(( - intersection == - HalfIntervals{{{0, 1}, {3, 18446744073709551615UL}}} - )); - } - - { - auto left = HalfIntervals{{{1, 2}, {2, 3}, {3, 4}, {4, 5}}}; - - ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 4}}}).convertToMarkRangesFinal().size(), 3); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 5}}}).convertToMarkRangesFinal().size(), 4); - } - - { - auto left = HalfIntervals{{{1, 3}, {3, 5}, {5, 7}}}; - - ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 5}}}).convertToMarkRangesFinal().size(), 1); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 7}}}).convertToMarkRangesFinal().size(), 2); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 6}}}).convertToMarkRangesFinal().size(), 2); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 7}}}).convertToMarkRangesFinal().size(), 3); - } - - { - auto left = HalfIntervals{{{1, 3}}}; - - ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 4}}}).convertToMarkRangesFinal().size(), 0); - } - - { - auto left = HalfIntervals{{{1, 2}, {3, 4}, {5, 6}}}; - - ASSERT_EQ(getIntersection(left, HalfIntervals{{{2, 3}}}).convertToMarkRangesFinal().size(), 0); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 5}}}).convertToMarkRangesFinal().size(), 0); - ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 6}}}).convertToMarkRangesFinal().size(), 3); - } -} - -TEST(HalfIntervals, TwoRequests) -{ - auto left = HalfIntervals{{{1, 2}, {2, 3}}}; - auto right = HalfIntervals{{{2, 3}, {3, 4}}}; - auto intersection = left.intersect(right); - - ASSERT_TRUE(( - intersection == - HalfIntervals{{{2, 3}}} - )); - - /// With negation - left = HalfIntervals{{{1, 2}, {2, 3}}}.negate(); - right = HalfIntervals{{{2, 3}, {3, 4}}}; - intersection = left.intersect(right); - - - ASSERT_TRUE(( - intersection == - HalfIntervals{{{3, 4}}} - )); -} - -TEST(HalfIntervals, SelfIntersection) -{ - auto left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}}; - auto right = left; - auto intersection = left.intersect(right); - - ASSERT_TRUE(( - intersection == right - )); - - left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}}; - right = left; - right.negate(); - intersection = left.intersect(right); - - ASSERT_TRUE(( - intersection == HalfIntervals{} - )); -} - - -TEST(Coordinator, Simple) -{ - PartitionReadRequest request; - request.partition_id = "a"; - request.part_name = "b"; - request.projection_name = "c"; - request.block_range = PartBlockRange{1, 2}; - request.mark_ranges = MarkRanges{{1, 2}, {3, 4}}; - - ParallelReplicasReadingCoordinator coordinator; - auto response = coordinator.handleRequest(request); - - ASSERT_FALSE(response.denied) << "Process request at first has to be accepted"; - - ASSERT_EQ(response.mark_ranges.size(), request.mark_ranges.size()); - - for (int i = 0; i < response.mark_ranges.size(); ++i) - EXPECT_EQ(response.mark_ranges[i], request.mark_ranges[i]); - - response = coordinator.handleRequest(request); - ASSERT_TRUE(response.denied) << "Process the same request second time"; -} - - -TEST(Coordinator, TwoRequests) -{ - PartitionReadRequest first; - first.partition_id = "a"; - first.part_name = "b"; - first.projection_name = "c"; - first.block_range = PartBlockRange{0, 0}; - first.mark_ranges = MarkRanges{{1, 2}, {2, 3}}; - - auto second = first; - second.mark_ranges = MarkRanges{{2, 3}, {3, 4}}; - - ParallelReplicasReadingCoordinator coordinator; - auto response = coordinator.handleRequest(first); - - ASSERT_FALSE(response.denied) << "First request must me accepted"; - - ASSERT_EQ(response.mark_ranges.size(), first.mark_ranges.size()); - for (int i = 0; i < response.mark_ranges.size(); ++i) - EXPECT_EQ(response.mark_ranges[i], first.mark_ranges[i]); - - response = coordinator.handleRequest(second); - ASSERT_FALSE(response.denied); - ASSERT_EQ(response.mark_ranges.size(), 1); - ASSERT_EQ(response.mark_ranges.front(), (MarkRange{3, 4})); -} - - -TEST(Coordinator, PartIntersections) -{ - { - PartSegments boundaries; - - boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{4, 4}, {"TestPart", "TestProjection"}}); - - ASSERT_EQ(boundaries.getIntersectionResult({{1, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - - boundaries.addPart(PartToRead{{5, 5}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{0, 0}, {"TestPart", "TestProjection"}}); - - ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{0, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - } - - { - PartSegments boundaries; - boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{4, 5}, {"TestPart", "TestProjection"}}); - - ASSERT_EQ(boundaries.getIntersectionResult({{2, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{0, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - } - - { - PartSegments boundaries; - boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{4, 6}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{7, 9}, {"TestPart", "TestProjection"}}); - - ASSERT_EQ(boundaries.getIntersectionResult({{2, 8}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{4, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{3, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{5, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - } - - { - PartSegments boundaries; - - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{0, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); - - boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}}); - boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}}); - - ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); - ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); - ASSERT_EQ(boundaries.getIntersectionResult({{100, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); - } -} diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 9e29d438a4b..40ea84ec68b 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -1,15 +1,16 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include #include +#include +#include +#include #include @@ -207,6 +208,8 @@ struct SelectQueryInfo /// Configured in StorageDistributed::getQueryProcessingStage() ClusterPtr optimized_cluster; + mutable ParallelReplicasReadingCoordinatorPtr coordinator; + TreeRewriterResultPtr syntax_analyzer_result; /// This is an additional filer applied to current table. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 740ad67cc95..8da9e74a9f4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -140,52 +140,6 @@ namespace ActionLocks namespace { -/// select query has database, table and table function names as AST pointers -/// Creates a copy of query, changes database, table and table function names. -ASTPtr rewriteSelectQuery( - ContextPtr context, - const ASTPtr & query, - const std::string & remote_database, - const std::string & remote_table, - ASTPtr table_function_ptr = nullptr) -{ - auto modified_query_ast = query->clone(); - - ASTSelectQuery & select_query = modified_query_ast->as(); - - // Get rid of the settings clause so we don't send them to remote. Thus newly non-important - // settings won't break any remote parser. It's also more reasonable since the query settings - // are written into the query context and will be sent by the query pipeline. - select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {}); - - if (table_function_ptr) - select_query.addTableFunction(table_function_ptr); - else - select_query.replaceDatabaseAndTable(remote_database, remote_table); - - /// Restore long column names (cause our short names are ambiguous). - /// TODO: aliased table functions & CREATE TABLE AS table function cases - if (!table_function_ptr) - { - RestoreQualifiedNamesVisitor::Data data; - data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as(), 0)); - data.remote_table.database = remote_database; - data.remote_table.table = remote_table; - RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); - } - - /// To make local JOIN works, default database should be added to table names. - /// But only for JOIN section, since the following should work using default_database: - /// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value - /// (see 01487_distributed_in_not_default_db) - AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(), - /* only_replace_current_database_function_= */false, - /* only_replace_in_join_= */true); - visitor.visit(modified_query_ast); - - return modified_query_ast; -} - /// Calculate maximum number in file names in directory and all subdirectories. /// To ensure global order of data blocks yet to be sent across server restarts. UInt64 getMaximumFileNumber(const std::string & dir_path) @@ -696,6 +650,7 @@ void StorageDistributed::read( const size_t /*max_block_size*/, const size_t /*num_streams*/) { + const auto * select_query = query_info.query->as(); if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas) throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature"); @@ -719,10 +674,11 @@ void StorageDistributed::read( query_ast = query_info.query; } - auto modified_query_ast = rewriteSelectQuery( - local_context, query_ast, + const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( + local_context, query_info.query, remote_database, remote_table, remote_table_function_ptr); + /// Return directly (with correct header) if no shard to query. if (query_info.getCluster()->getShardsInfo().empty()) { @@ -746,25 +702,13 @@ void StorageDistributed::read( storage_snapshot, processed_stage); - - auto settings = local_context->getSettingsRef(); - bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests; - - if (parallel_replicas) - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, main_table, remote_table_function_ptr, - select_stream_factory, modified_query_ast, - local_context, query_info, - sharding_key_expr, sharding_key_column_name, - query_info.cluster, processed_stage); - else - ClusterProxy::executeQuery( - query_plan, header, processed_stage, - main_table, remote_table_function_ptr, - select_stream_factory, log, modified_query_ast, - local_context, query_info, - sharding_key_expr, sharding_key_column_name, - query_info.cluster); + ClusterProxy::executeQuery( + query_plan, header, processed_stage, + main_table, remote_table_function_ptr, + select_stream_factory, log, modified_query_ast, + local_context, query_info, + sharding_key_expr, sharding_key_column_name, + query_info.cluster); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4ef34ae91d5..b003a111398 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1,4 +1,5 @@ #include "StorageMergeTree.h" +#include "Core/QueryProcessingStage.h" #include "Storages/MergeTree/IMergeTreeDataPart.h" #include @@ -14,6 +15,8 @@ #include #include #include +#include +#include #include #include #include @@ -209,15 +212,39 @@ void StorageMergeTree::read( size_t max_block_size, size_t num_streams) { - /// If true, then we will ask initiator if we can read chosen ranges - bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; + if (local_context->canUseParallelReplicasOnInitiator()) + { + auto table_id = getStorageID(); - if (enable_parallel_reading) - LOG_TRACE(log, "Parallel reading from replicas enabled: {}", enable_parallel_reading); + const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( + local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - if (auto plan = reader.read( - column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading)) - query_plan = std::move(*plan); + auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + + Block header = + InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + + ClusterProxy::SelectStreamFactory select_stream_factory = + ClusterProxy::SelectStreamFactory( + header, + {}, + storage_snapshot, + processed_stage); + + ClusterProxy::executeQueryWithParallelReplicas( + query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, + select_stream_factory, modified_query_ast, + local_context, query_info, cluster); + } + else + { + if (auto plan = reader.read( + column_names, storage_snapshot, query_info, + local_context, max_block_size, num_streams, + processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower())) + query_plan = std::move(*plan); + } /// Now, copy of parts that is required for the query, stored in the processors, /// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning, diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 2afd9e8a63b..b31707eeb62 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -162,4 +162,3 @@ public: } - diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7e728d3f21e..de2e9ec4698 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -71,15 +71,18 @@ #include #include -#include -#include +#include +#include #include #include -#include -#include +#include #include +#include +#include +#include #include + #include #include #include @@ -4543,9 +4546,6 @@ void StorageReplicatedMergeTree::read( const size_t max_block_size, const size_t num_streams) { - /// If true, then we will ask initiator if we can read chosen ranges - const bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; - SCOPE_EXIT({ /// Now, copy of parts that is required for the query, stored in the processors, /// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning, @@ -4564,16 +4564,43 @@ void StorageReplicatedMergeTree::read( auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); if (auto plan = reader.read( column_names, storage_snapshot, query_info, local_context, - max_block_size, num_streams, processed_stage, std::move(max_added_blocks), enable_parallel_reading)) + max_block_size, num_streams, processed_stage, std::move(max_added_blocks), /*enable_parallel_reading*/false)) query_plan = std::move(*plan); return; } - if (auto plan = reader.read( - column_names, storage_snapshot, query_info, local_context, - max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading)) + if (local_context->canUseParallelReplicasOnInitiator()) { - query_plan = std::move(*plan); + auto table_id = getStorageID(); + + const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( + local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + + auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + + Block header = + InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + + ClusterProxy::SelectStreamFactory select_stream_factory = + ClusterProxy::SelectStreamFactory( + header, + {}, + storage_snapshot, + processed_stage); + + ClusterProxy::executeQueryWithParallelReplicas( + query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, + select_stream_factory, modified_query_ast, + local_context, query_info, cluster); + } + else + { + if (auto plan = reader.read( + column_names, storage_snapshot, query_info, + local_context, max_block_size, num_streams, + processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower())) + query_plan = std::move(*plan); } } diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 0721cfaa9c4..b2737249166 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -79,7 +79,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( ParserExpression expr_parser; - while (Block current = executor.read()) + while (Block current = executor.readBlock()) { ColumnPtr name = current.getByName("name").column; ColumnPtr type = current.getByName("type").column; @@ -187,7 +187,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( executor.setMainTable(remote_table_id); ColumnsDescription res; - while (auto block = executor.read()) + while (auto block = executor.readBlock()) { const auto & name_col = *block.getByName("name").column; const auto & type_col = *block.getByName("type").column; diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c77acfb679f..b671b8683d7 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -209,6 +209,26 @@ CI_CONFIG = { "Stateful tests (release, DatabaseReplicated)": { "required_build": "package_release", }, + # Stateful tests for parallel replicas + "Stateful tests (release, ParallelReplicas)": { + "required_build": "package_release", + }, + "Stateful tests (debug, ParallelReplicas)": { + "required_build": "package_debug", + }, + "Stateful tests (asan, ParallelReplicas)": { + "required_build": "package_asan", + }, + "Stateful tests (msan, ParallelReplicas)": { + "required_build": "package_msan", + }, + "Stateful tests (ubsan, ParallelReplicas)": { + "required_build": "package_ubsan", + }, + "Stateful tests (tsan, ParallelReplicas)": { + "required_build": "package_tsan", + }, + # End stateful tests for parallel replicas "Stateless tests (asan)": { "required_build": "package_asan", }, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index cf5f53afbf9..c33454d1d90 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -48,7 +48,8 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total): result.append("USE_DATABASE_ORDINARY=1") if "wide parts enabled" in check_name: result.append("USE_POLYMORPHIC_PARTS=1") - + if "ParallelReplicas" in check_name: + result.append("USE_PARALLEL_REPLICAS=1") if "s3 storage" in check_name: result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") @@ -355,16 +356,34 @@ def main(): print(f"::notice:: {check_name} Report url: {report_url}") if args.post_commit_status == "commit_status": - post_commit_status( - gh, pr_info.sha, check_name_with_group, description, state, report_url - ) + if "parallelreplicas" in check_name.lower(): + post_commit_status( + gh, + pr_info.sha, + check_name_with_group, + description, + "success", + report_url, + ) + else: + post_commit_status( + gh, pr_info.sha, check_name_with_group, description, state, report_url + ) elif args.post_commit_status == "file": - post_commit_status_to_file( - post_commit_path, - description, - state, - report_url, - ) + if "parallelreplicas" in check_name.lower(): + post_commit_status_to_file( + post_commit_path, + description, + "success", + report_url, + ) + else: + post_commit_status_to_file( + post_commit_path, + description, + state, + report_url, + ) else: raise Exception( f'Unknown post_commit_status option "{args.post_commit_status}"' @@ -382,7 +401,11 @@ def main(): ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) if state != "success": - if FORCE_TESTS_LABEL in pr_info.labels: + # Parallel replicas are always green for now + if ( + FORCE_TESTS_LABEL in pr_info.labels + or "parallelreplicas" in check_name.lower() + ): print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") else: sys.exit(1) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 50d940bc23c..366197cfd03 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -442,6 +442,7 @@ class FailureReason(enum.Enum): STRESS = "stress" BUILD = "not running for current build" BACKWARD_INCOMPATIBLE = "test is backward incompatible" + NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -729,6 +730,9 @@ class TestCase: ): return FailureReason.DISABLED + elif "no-parallel-replicas" in tags and args.no_parallel_replicas: + return FailureReason.NO_PARALLEL_REPLICAS + elif args.skip and any(s in self.name for s in args.skip): return FailureReason.SKIP @@ -2399,6 +2403,13 @@ if __name__ == "__main__": default=False, help="Report statistics about log messages", ) + parser.add_argument( + "--no-parallel-replicas", + action="store_true", + default=False, + help="Do not include tests that are not supported with parallel replicas feature", + ) + args = parser.parse_args() if args.queries and not os.path.isdir(args.queries): diff --git a/tests/performance/memory_bound_merging.xml b/tests/performance/memory_bound_merging.xml index 3b13400151c..15dc1b29fba 100644 --- a/tests/performance/memory_bound_merging.xml +++ b/tests/performance/memory_bound_merging.xml @@ -11,7 +11,5 @@ select avg(a) from remote('127.0.0.{{1,2}}', default, t_mbm) group by a format Null - select * from remote('127.0.0.{{1,2}}', default, t_mbm) group by a format Null settings allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, use_hedged_requests = 0 - drop table t_mbm diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index 47d3470ef6e..98e53cd50ab 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -98,8 +98,9 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off 502 2000 503 2000 504 2000 +1000000 -- { echoOn } -- -explain pipeline select a from dist_pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500; (Expression) ExpressionTransform (Limit) @@ -112,28 +113,29 @@ ExpressionTransform (Expression) ExpressionTransform × 4 (MergingAggregated) - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - FinishAggregatingInOrderTransform 3 → 1 - (Union) - (Aggregating) - SortingAggregatedForMemoryBoundMergingTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - FinishAggregatingInOrderTransform 4 → 1 - AggregatingInOrderTransform × 4 - (Expression) - ExpressionTransform × 4 - (ReadFromMergeTree) - MergeTreeInOrder × 4 0 → 1 - (ReadFromRemoteParallelReplicas) -select a, count() from dist_pr_t group by a order by a limit 5 offset 500; + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 6 → 1 + (Union) + (Aggregating) + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + FinishAggregatingInOrderTransform 4 → 1 + AggregatingInOrderTransform × 4 + (Expression) + ExpressionTransform × 4 + (ReadFromMergeTree) + MergeTreeInOrder × 4 0 → 1 + (ReadFromRemoteParallelReplicas) +select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 502 1000 503 1000 504 1000 -select a, count() from dist_pr_t group by a, b order by a limit 5 offset 500; +select a, count() from pr_t group by a, b order by a limit 5 offset 500; 500 1000 501 1000 502 1000 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index f4a1e75e398..a38e4c5ec6b 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -1,13 +1,13 @@ -- Tags: no-parallel drop table if exists pr_t; -drop table if exists dist_pr_t; drop table if exists dist_t_different_dbs; drop table if exists shard_1.t_different_dbs; drop table if exists t_different_dbs; drop table if exists dist_t; drop table if exists t; + create table t(a UInt64, b UInt64) engine=MergeTree order by a; system stop merges t; insert into t select number, number from numbers_mt(1e6); @@ -15,6 +15,7 @@ insert into t select number, number from numbers_mt(1e6); set enable_memory_bound_merging_of_aggregation_results = 1; set max_threads = 4; set optimize_aggregation_in_order = 1; +set optimize_read_in_order = 1; set prefer_localhost_replica = 1; -- slightly different transforms will be generated by reading steps if we let settings randomisation to change this setting value -- @@ -56,26 +57,28 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off -- { echoOff } -- +create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a; +insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); + set allow_experimental_parallel_reading_from_replicas = 1; set max_parallel_replicas = 3; set use_hedged_requests = 0; +set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +set distributed_aggregation_memory_efficient=1; -create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a; -insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); -create table dist_pr_t as pr_t engine = Distributed(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), pr_t); +select count() from pr_t; -- { echoOn } -- -explain pipeline select a from dist_pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500; -select a, count() from dist_pr_t group by a order by a limit 5 offset 500; -select a, count() from dist_pr_t group by a, b order by a limit 5 offset 500; +select a, count() from pr_t group by a order by a limit 5 offset 500; +select a, count() from pr_t group by a, b order by a limit 5 offset 500; -- { echoOff } -- -drop table pr_t; -drop table dist_pr_t; -drop table dist_t_different_dbs; -drop table shard_1.t_different_dbs; -drop table t_different_dbs; -drop table dist_t; -drop table t; +drop table if exists pr_t; +drop table if exists dist_t_different_dbs; +drop table if exists shard_1.t_different_dbs; +drop table if exists t_different_dbs; +drop table if exists dist_t; +drop table if exists t; diff --git a/tests/queries/1_stateful/00009_uniq_distributed.sql b/tests/queries/1_stateful/00009_uniq_distributed.sql index f78604fd401..352514cd059 100644 --- a/tests/queries/1_stateful/00009_uniq_distributed.sql +++ b/tests/queries/1_stateful/00009_uniq_distributed.sql @@ -1,3 +1,4 @@ -- Tags: distributed + SELECT uniq(UserID), uniqIf(UserID, CounterID = 800784), uniqIf(FUniqID, RegionID = 213) FROM remote('127.0.0.{1,2}', test, hits) diff --git a/tests/queries/1_stateful/00012_sorting_distributed.sql b/tests/queries/1_stateful/00012_sorting_distributed.sql index 2f852af1dba..afbaf89d9ae 100644 --- a/tests/queries/1_stateful/00012_sorting_distributed.sql +++ b/tests/queries/1_stateful/00012_sorting_distributed.sql @@ -1,3 +1,4 @@ -- Tags: distributed + SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1,2}', test, hits) ORDER BY EventTime DESC LIMIT 10 diff --git a/tests/queries/1_stateful/00013_sorting_of_nested.sql b/tests/queries/1_stateful/00013_sorting_of_nested.sql index 44f7684d746..f97120e2b98 100644 --- a/tests/queries/1_stateful/00013_sorting_of_nested.sql +++ b/tests/queries/1_stateful/00013_sorting_of_nested.sql @@ -1,2 +1,4 @@ +-- Tags: no-parallel-replicas + SELECT ParsedParams.Key1 FROM test.visits FINAL WHERE VisitID != 0 AND notEmpty(ParsedParams.Key1) ORDER BY VisitID LIMIT 10 diff --git a/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.sql b/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.sql index 2afe28639f2..50a3402244e 100644 --- a/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.sql +++ b/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.sql @@ -1,3 +1,4 @@ -- Tags: distributed + SELECT anyIf(SearchPhrase, CounterID = -1) FROM remote('127.0.0.{1,2}:9000', test, hits) diff --git a/tests/queries/1_stateful/00022_merge_prewhere.sql b/tests/queries/1_stateful/00022_merge_prewhere.sql index 74a3677b68e..400a896d5a8 100644 --- a/tests/queries/1_stateful/00022_merge_prewhere.sql +++ b/tests/queries/1_stateful/00022_merge_prewhere.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + DROP TABLE IF EXISTS test.merge_hits; CREATE TABLE IF NOT EXISTS test.merge_hits AS test.hits ENGINE = Merge(test, '^hits$'); SELECT count() FROM test.merge_hits WHERE AdvEngineID = 2; diff --git a/tests/queries/1_stateful/00042_any_left_join.sql b/tests/queries/1_stateful/00042_any_left_join.sql index b87cf88f007..c7c0f0f987a 100644 --- a/tests/queries/1_stateful/00042_any_left_join.sql +++ b/tests/queries/1_stateful/00042_any_left_join.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + SELECT EventDate, hits, diff --git a/tests/queries/1_stateful/00043_any_left_join.sql b/tests/queries/1_stateful/00043_any_left_join.sql index 704d38f727a..6b8cce54051 100644 --- a/tests/queries/1_stateful/00043_any_left_join.sql +++ b/tests/queries/1_stateful/00043_any_left_join.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + SELECT EventDate, count() AS hits, diff --git a/tests/queries/1_stateful/00044_any_left_join_string.sql b/tests/queries/1_stateful/00044_any_left_join_string.sql index a4f2e9e1b96..ceb7a1c1783 100644 --- a/tests/queries/1_stateful/00044_any_left_join_string.sql +++ b/tests/queries/1_stateful/00044_any_left_join_string.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + SELECT domain, hits, diff --git a/tests/queries/1_stateful/00063_loyalty_joins.sql b/tests/queries/1_stateful/00063_loyalty_joins.sql index 1e7011ea909..44f0767a87a 100644 --- a/tests/queries/1_stateful/00063_loyalty_joins.sql +++ b/tests/queries/1_stateful/00063_loyalty_joins.sql @@ -1,15 +1,17 @@ +-- Tags: no-parallel-replicas + SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; SELECT - loyalty, + loyalty, count() -FROM test.hits ANY LEFT JOIN +FROM test.hits ANY LEFT JOIN ( SELECT - UserID, - sum(SearchEngineID = 2) AS yandex, - sum(SearchEngineID = 3) AS google, + UserID, + sum(SearchEngineID = 2) AS yandex, + sum(SearchEngineID = 3) AS google, toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM test.hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) @@ -21,18 +23,18 @@ ORDER BY loyalty ASC; SELECT - loyalty, + loyalty, count() FROM ( SELECT UserID FROM test.hits -) ANY LEFT JOIN +) ANY LEFT JOIN ( SELECT - UserID, - sum(SearchEngineID = 2) AS yandex, - sum(SearchEngineID = 3) AS google, + UserID, + sum(SearchEngineID = 2) AS yandex, + sum(SearchEngineID = 3) AS google, toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM test.hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) @@ -44,23 +46,23 @@ ORDER BY loyalty ASC; SELECT - loyalty, + loyalty, count() FROM ( SELECT - loyalty, + loyalty, UserID FROM ( SELECT UserID FROM test.hits - ) ANY LEFT JOIN + ) ANY LEFT JOIN ( SELECT - UserID, - sum(SearchEngineID = 2) AS yandex, - sum(SearchEngineID = 3) AS google, + UserID, + sum(SearchEngineID = 2) AS yandex, + sum(SearchEngineID = 3) AS google, toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM test.hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) @@ -73,19 +75,19 @@ ORDER BY loyalty ASC; SELECT - loyalty, - count() AS c, + loyalty, + count() AS c, bar(log(c + 1) * 1000, 0, log(3000000) * 1000, 80) -FROM test.hits ANY INNER JOIN +FROM test.hits ANY INNER JOIN ( SELECT - UserID, + UserID, toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM ( SELECT - UserID, - sum(SearchEngineID = 2) AS yandex, + UserID, + sum(SearchEngineID = 2) AS yandex, sum(SearchEngineID = 3) AS google FROM test.hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index 515a2410583..35f0c7b60b9 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + USE test; DROP TABLE IF EXISTS join; @@ -7,7 +9,7 @@ INSERT INTO join SELECT UserID, toInt8(if((sum(SearchEngineID = 2) AS yandex) > (sum(SearchEngineID = 3) AS google), - yandex / (yandex + google), + yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty FROM hits WHERE (SearchEngineID = 2) OR (SearchEngineID = 3) diff --git a/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.sql b/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.sql index c7a34c493c9..c60e342dd41 100644 --- a/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.sql +++ b/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.sql @@ -1,4 +1,5 @@ -- Tags: replica, distributed, no-random-settings + SET max_parallel_replicas = 2; SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10 diff --git a/tests/queries/1_stateful/00074_full_join.sql b/tests/queries/1_stateful/00074_full_join.sql index f049be2a74d..c1d9e4be1a4 100644 --- a/tests/queries/1_stateful/00074_full_join.sql +++ b/tests/queries/1_stateful/00074_full_join.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + set any_join_distinct_right_table_keys = 1; set joined_subquery_requires_alias = 0; diff --git a/tests/queries/1_stateful/00075_left_array_join.sql b/tests/queries/1_stateful/00075_left_array_join.sql index 1fd045a26bf..3540d791157 100644 --- a/tests/queries/1_stateful/00075_left_array_join.sql +++ b/tests/queries/1_stateful/00075_left_array_join.sql @@ -1,2 +1,4 @@ +-- Tags: no-parallel-replicas + SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; SELECT UserID, EventTime::DateTime('Asia/Dubai'), pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; diff --git a/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql b/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql index 8e6742bb1e1..9431e1cf596 100644 --- a/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql +++ b/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + SELECT PP.Key1 AS `ym:s:paramsLevel1`, sum(arrayAll(`x_1` -> `x_1`= '', ParsedParams.Key2)) AS `ym:s:visits` FROM test.hits ARRAY JOIN ParsedParams AS `PP` WHERE CounterID = 1704509 GROUP BY `ym:s:paramsLevel1` ORDER BY PP.Key1, `ym:s:visits` LIMIT 0, 100; SELECT PP.Key1 AS x1, ParsedParams.Key2 AS x2 FROM test.hits ARRAY JOIN ParsedParams AS PP WHERE CounterID = 1704509 ORDER BY x1, x2 LIMIT 10; SELECT ParsedParams.Key2 AS x FROM test.hits ARRAY JOIN ParsedParams AS PP ORDER BY x DESC LIMIT 10; diff --git a/tests/queries/1_stateful/00080_array_join_and_union.sql b/tests/queries/1_stateful/00080_array_join_and_union.sql index d9aa1cc17cc..2f2e5e9324f 100644 --- a/tests/queries/1_stateful/00080_array_join_and_union.sql +++ b/tests/queries/1_stateful/00080_array_join_and_union.sql @@ -1 +1,3 @@ +-- Tags: no-parallel-replicas + SELECT count() FROM (SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10 UNION ALL SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10); diff --git a/tests/queries/1_stateful/00084_external_aggregation.sql b/tests/queries/1_stateful/00084_external_aggregation.sql index 816d95f4b8b..330aa158cf7 100644 --- a/tests/queries/1_stateful/00084_external_aggregation.sql +++ b/tests/queries/1_stateful/00084_external_aggregation.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings +-- Tags: no-random-settings, no-parallel-replicas SET max_bytes_before_external_group_by = 200000000; diff --git a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql index 1e476d3a27d..745bb125c2b 100644 --- a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql +++ b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -1,3 +1,6 @@ +-- Tags: no-parallel-replicas +-- Requires investigation (max_bytes_to_read is not respected) + SET max_bytes_to_read = 600000000; SET optimize_move_to_prewhere = 1; diff --git a/tests/queries/1_stateful/00092_obfuscator.sh b/tests/queries/1_stateful/00092_obfuscator.sh index 85f476c6ae5..f19473f01ac 100755 --- a/tests/queries/1_stateful/00092_obfuscator.sh +++ b/tests/queries/1_stateful/00092_obfuscator.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-parallel-replicas +# clickhouse-local may not work with parallel replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/1_stateful/00096_obfuscator_save_load.sh b/tests/queries/1_stateful/00096_obfuscator_save_load.sh index a88dfcdb9b9..1bb212e1bba 100755 --- a/tests/queries/1_stateful/00096_obfuscator_save_load.sh +++ b/tests/queries/1_stateful/00096_obfuscator_save_load.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-parallel-replicas +# clickhouse-local may not work with parallel replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index fd3fde7636d..2cab6e70d22 100644 --- a/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel-replicas + SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; diff --git a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql index 6f910646fb7..5d2476226ba 100644 --- a/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql +++ b/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql @@ -1,4 +1,4 @@ --- Tags: distributed +-- Tags: distributed, no-parallel-replicas SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID); SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1; diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 294d71b384b..35483149498 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan, no-replicated-database, no-parallel +-- Tags: no-tsan, no-replicated-database, no-parallel, no-parallel-replicas -- Tag no-replicated-database: Fails due to additional replicas or shards DROP TABLE IF EXISTS fixed_granularity_table; diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql index e325c18200b..32079111f6c 100644 --- a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql @@ -1,3 +1,6 @@ +-- Tags: no-parallel-replicas +-- Merge tables doesn't work with parallel replicas currently + SET max_execution_speed = 4000000, timeout_before_checking_execution_speed = 0; CREATE TEMPORARY TABLE times (t DateTime); diff --git a/tests/queries/1_stateful/00166_explain_estimate.sql b/tests/queries/1_stateful/00166_explain_estimate.sql index c4071271736..abac92ecb2e 100644 --- a/tests/queries/1_stateful/00166_explain_estimate.sql +++ b/tests/queries/1_stateful/00166_explain_estimate.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database +-- Tags: no-replicated-database, no-parallel-replicas -- Tag no-replicated-database: Requires investigation EXPLAIN ESTIMATE SELECT count() FROM test.hits WHERE CounterID = 29103473; diff --git a/tests/queries/1_stateful/00170_s3_cache.sql b/tests/queries/1_stateful/00170_s3_cache.sql index b03b2a16bf0..81592255428 100644 --- a/tests/queries/1_stateful/00170_s3_cache.sql +++ b/tests/queries/1_stateful/00170_s3_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-random-settings +-- Tags: no-parallel, no-random-settings, no-parallel-replicas -- { echo } diff --git a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql index 7068780a1b1..07788af927e 100644 --- a/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql +++ b/tests/queries/1_stateful/00171_grouping_aggregated_transform_bug.sql @@ -1,4 +1,4 @@ --- Tags: distributed +-- Tags: distributed, no-parallel-replicas SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS max_block_size = 63169; SELECT sum(cityHash64(*)) FROM (SELECT CounterID, quantileTiming(0.5)(SendTiming), count() FROM remote('127.0.0.{1,2,3,4,5,6,7,8,9,10}', test.hits) WHERE SendTiming != -1 GROUP BY CounterID) SETTINGS optimize_aggregation_in_order = 1, max_block_size = 63169; diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index cc3d2274ecd..b31e418b492 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -1 +1,3 @@ +-- Tags: no-parallel-replicas + EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1; diff --git a/tests/queries/1_stateful/00172_hits_joins.sql.j2 b/tests/queries/1_stateful/00172_hits_joins.sql.j2 index 4599d1d5a5d..4617fe5aef8 100644 --- a/tests/queries/1_stateful/00172_hits_joins.sql.j2 +++ b/tests/queries/1_stateful/00172_hits_joins.sql.j2 @@ -1,3 +1,4 @@ +-- Tags: no-parallel-replicas {% for join_algorithm in ['hash', 'parallel_hash', 'full_sorting_merge', 'grace_hash'] -%} SET max_rows_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% endif %}'; diff --git a/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh index 771c7ab5436..0b308c65061 100755 --- a/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh +++ b/tests/queries/1_stateful/00175_obfuscator_schema_inference.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel-replicas CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 26a6c5a25bd8553c89d7bb89387d4e79887bb8d3 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Feb 2023 09:13:48 -0500 Subject: [PATCH 75/78] combine guide and reference for lightweight deletes --- docs/en/sql-reference/statements/delete.md | 35 ++++++++++++++++++++-- 1 file changed, 33 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 0acb6637ea6..81a0c57f262 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -10,7 +10,7 @@ sidebar_label: DELETE DELETE FROM [db.]table [ON CLUSTER cluster] [WHERE expr] ``` -`DELETE FROM` removes rows from table `[db.]table` that match expression `expr`. The deleted rows are marked as deleted immediately and will be automatically filtered out of all subsequent queries. Cleanup of data happens asynchronously in background. This feature is only available for MergeTree table engine family. +`DELETE FROM` removes rows from the table `[db.]table` that match the expression `expr`. The deleted rows are marked as deleted immediately and will be automatically filtered out of all subsequent queries. Cleanup of data happens asynchronously in the background. This feature is only available for the MergeTree table engine family. For example, the following query deletes all rows from the `hits` table where the `Title` column contains the text `hello`: @@ -32,7 +32,7 @@ SET allow_experimental_lightweight_delete = true; An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. :::warning -Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on an OLTP system. Ligthweight deletes are currently efficient for wide parts, but for compact parts they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. +Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on an OLTP system. Lightweight deletes are currently efficient for wide parts, but for compact parts, they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. ::: :::note @@ -41,3 +41,34 @@ Even though deletes are becoming more lightweight in ClickHouse, they should sti grant ALTER DELETE ON db.table to username; ``` ::: + +## Lightweight Delete Internals + +The idea behind Lightweight Delete is that when a `DELETE FROM table ...` query is executed ClickHouse only saves a mask where each row is marked as either “existing” or as “deleted”. Those “deleted” rows become invisible for subsequent queries, but physically the rows are removed only later by subsequent merges. Writing this mask is usually much more lightweight than what is done by `ALTER table DELETE ...` query. + +### How it is implemented +The mask is implemented as a hidden `_row_exists` system column that stores True for all visible rows and False for deleted ones. This column is only present in a part if some rows in this part were deleted. In other words, the column is not persisted when it has all values equal to True. + +## SELECT query +When the column is present `SELECT ... FROM table WHERE condition` query internally is extended by an additional predicate on `_row_exists` and becomes similar to +```sql + SELECT ... FROM table PREWHERE _row_exists WHERE condition +``` +At execution time the column `_row_exists` is read to figure out which rows are not visible and if there are many deleted rows it can figure out which granules can be fully skipped when reading the rest of the columns. + +## DELETE query +`DELETE FROM table WHERE condition` is translated into `ALTER table UPDATE _row_exists = 0 WHERE condition` mutation. Internally this mutation is executed in 2 steps: +1. `SELECT count() FROM table WHERE condition` for each individual part to figure out if the part is affected. +2. Mutate affected parts, and make hardlinks for unaffected parts. Mutating a part in fact only writes `_row_exists` column and just hardlinks all other columns’ files in the case of Wide parts. But for Compact parts, all columns are rewritten because they all are stored together in one file. + +So if we compare Lightweight Delete to `ALTER DELETE` in the first step they both do the same thing to figure out which parts are affected, but in the second step `ALTER DELETE` does much more work because it reads and rewrites all columns’ files for the affected parts. + +With the described implementation now we can see what can negatively affect 'DELETE FROM' execution time: +- Heavy WHERE condition in DELETE query +- Mutations queue filled with other mutations, because all mutations on a table are executed sequentially +- Table having a very large number of data parts +- Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file. + +:::note +This implementation might change in the future. +::: From 7889b632d680dadad8e6d482133381cb9d95d68f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Feb 2023 09:16:33 -0500 Subject: [PATCH 76/78] add metadata --- docs/en/sql-reference/statements/delete.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 81a0c57f262..82849f7768c 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -2,6 +2,8 @@ slug: /en/sql-reference/statements/delete sidebar_position: 36 sidebar_label: DELETE +description: Lightweight deletes simplify the process of deleting data from the database. +keywords: [delete] --- # DELETE Statement From c9244335ef5877be732da06cb164dc2a28dd144a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Feb 2023 09:21:30 -0500 Subject: [PATCH 77/78] move title to frontmatter --- docs/en/sql-reference/statements/delete.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 82849f7768c..e1987e50af4 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -4,10 +4,9 @@ sidebar_position: 36 sidebar_label: DELETE description: Lightweight deletes simplify the process of deleting data from the database. keywords: [delete] +title: DELETE Statement --- -# DELETE Statement - ``` sql DELETE FROM [db.]table [ON CLUSTER cluster] [WHERE expr] ``` From 8464357bca943ff45c57ee89414dc9d138f37bb2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Feb 2023 13:57:47 -0500 Subject: [PATCH 78/78] fix heading level --- .../sql-reference/functions/comparison-functions.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/comparison-functions.md b/docs/en/sql-reference/functions/comparison-functions.md index ab19399bd4e..586c0dc54e6 100644 --- a/docs/en/sql-reference/functions/comparison-functions.md +++ b/docs/en/sql-reference/functions/comparison-functions.md @@ -21,14 +21,14 @@ For example, you can’t compare a date with a string. You have to use a functio Strings are compared by bytes. A shorter string is smaller than all strings that start with it and that contain at least one more character. -## equals, a = b and a == b operator +### equals, a `=` b and a `==` b operator -## notEquals, a != b and a \<\> b operator +### notEquals, a `!=` b and a `<>` b operator -## less, \< operator +### less, `<` operator -## greater, \> operator +### greater, `>` operator -## lessOrEquals, \<= operator +### lessOrEquals, `<=` operator -## greaterOrEquals, \>= operator +### greaterOrEquals, `>=` operator