From 548d79c2e80bb23f246c63fc7e33d0c01eb6b944 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 2 Mar 2023 12:31:09 +0000 Subject: [PATCH 001/226] Remove perf test duplicate_order_by_and_distinct.xml --- tests/performance/duplicate_order_by_and_distinct.xml | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 tests/performance/duplicate_order_by_and_distinct.xml diff --git a/tests/performance/duplicate_order_by_and_distinct.xml b/tests/performance/duplicate_order_by_and_distinct.xml deleted file mode 100644 index e36bc4705129..000000000000 --- a/tests/performance/duplicate_order_by_and_distinct.xml +++ /dev/null @@ -1,8 +0,0 @@ - - 1 - - - SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY EventDate, CounterID FORMAT Null - SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single) FORMAT Null - SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY toStartOfWeek(EventDate) FORMAT Null - From 1e7080a9aae4403ea613d401501f5b6498bf2df9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Apr 2023 17:35:47 +0000 Subject: [PATCH 002/226] ReadFromMergeTree: update sort description after applying prewhere info --- .../QueryPlan/ReadFromMergeTree.cpp | 66 +++++++++++-------- 1 file changed, 39 insertions(+), 27 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 291499ff412a..64a3a4c74ae3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -133,6 +133,35 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) return true; } +/// build sort description for output stream +static void updateSortDescriptionForOutputStream( + DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info) +{ + SortDescription sort_description; + const Block & header = output_stream.header; + for (const auto & column_name : sorting_key_columns) + { + if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) + == header.end()) + break; + sort_description.emplace_back(column_name, sort_direction); + } + if (!sort_description.empty()) + { + if (input_order_info) + { + output_stream.sort_scope = DataStream::SortScope::Stream; + const size_t used_prefix_of_sorting_key_size = input_order_info->used_prefix_of_sorting_key_size; + if (sort_description.size() > used_prefix_of_sorting_key_size) + sort_description.resize(used_prefix_of_sorting_key_size); + } + else + output_stream.sort_scope = DataStream::SortScope::Chunk; + } + + output_stream.sort_description = std::move(sort_description); +} + void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, const SelectQueryInfo & query_info_) const { @@ -244,33 +273,11 @@ ReadFromMergeTree::ReadFromMergeTree( /// Add explicit description. setStepDescription(data.getStorageID().getFullNameNotQuoted()); - { /// build sort description for output stream - SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); - const Block & header = output_stream->header; - const int sort_direction = getSortDirection(); - for (const auto & column_name : sorting_key_columns) - { - if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) - == header.end()) - break; - sort_description.emplace_back(column_name, sort_direction); - } - if (!sort_description.empty()) - { - if (query_info.getInputOrderInfo()) - { - output_stream->sort_scope = DataStream::SortScope::Stream; - const size_t used_prefix_of_sorting_key_size = query_info.getInputOrderInfo()->used_prefix_of_sorting_key_size; - if (sort_description.size() > used_prefix_of_sorting_key_size) - sort_description.resize(used_prefix_of_sorting_key_size); - } - else - output_stream->sort_scope = DataStream::SortScope::Chunk; - } - - output_stream->sort_description = std::move(sort_description); - } + updateSortDescriptionForOutputStream( + *output_stream, + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + getSortDirection(), + query_info.getInputOrderInfo()); } @@ -1425,6 +1432,11 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info prewhere_info_value, data.getPartitionValueType(), virt_column_names)}; + updateSortDescriptionForOutputStream( + *output_stream, + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + getSortDirection(), + query_info.getInputOrderInfo()); } bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() From 125e5c50b891038740a2a75e2570a201999c09f3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Apr 2023 21:30:03 +0000 Subject: [PATCH 003/226] allow to flush async insert queue --- programs/server/Server.cpp | 11 ++- src/Access/Common/AccessType.h | 1 + src/Core/Settings.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 71 +++++++++++++++---- src/Interpreters/AsynchronousInsertQueue.h | 11 ++- src/Interpreters/InterpreterSystemQuery.cpp | 17 +++++ src/Parsers/ASTSystemQuery.h | 1 + .../02726_async_insert_flush_queue.reference | 5 ++ .../02726_async_insert_flush_queue.sql | 28 ++++++++ 9 files changed, 128 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_queue.reference create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_queue.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8c0d50bae55c..cd08de126c97 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1461,16 +1461,21 @@ try /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); - const Settings & settings = global_context->getSettingsRef(); /// Initialize background executors after we load default_profile config. /// This is needed to load proper values of background_pool_size etc. global_context->initializeBackgroundExecutorsIfNeeded(); - if (settings.async_insert_threads) + size_t async_insert_threads = config().getUInt("async_insert_threads", 16); + bool async_insert_queue_flush_on_shutdown = config().getBool("async_insert_queue_flush_on_shutdown", false); + + if (async_insert_threads) + { global_context->setAsynchronousInsertQueue(std::make_shared( global_context, - settings.async_insert_threads)); + async_insert_threads, + async_insert_queue_flush_on_shutdown)); + } size_t mark_cache_size = server_settings.mark_cache_size; String mark_cache_policy = server_settings.mark_cache_policy; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 57fa75dc67b0..ae7e7ab5bf00 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -182,6 +182,7 @@ enum class AccessType M(SYSTEM_SYNC_FILE_CACHE, "SYNC FILE CACHE", GLOBAL, SYSTEM) \ M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \ + M(SYSTEM_FLUSH_ASYNC_INSERT_QUEUE, "FLUSH ASYNC INSERT QUEUE", GLOBAL, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \ M(SYSTEM_THREAD_FUZZER, "SYSTEM START THREAD FUZZER, SYSTEM STOP THREAD FUZZER, START THREAD FUZZER, STOP THREAD FUZZER", GLOBAL, SYSTEM) \ M(SYSTEM_UNFREEZE, "SYSTEM UNFREEZE", GLOBAL, SYSTEM) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 101f6f1f934b..96dbe26f820d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -646,7 +646,6 @@ class IColumn; M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ \ - M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ M(Bool, wait_for_async_insert, true, "If true wait for processing of asynchronous insertion", 0) \ M(Seconds, wait_for_async_insert_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "Timeout for waiting for processing asynchronous insertion", 0) \ @@ -783,6 +782,7 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_distributed_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, async_insert_threads, 16) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index b8de0246ae2b..0a817995eb49 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -128,9 +128,10 @@ void AsynchronousInsertQueue::InsertData::Entry::finish(std::exception_ptr excep } } -AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_) +AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_, bool flush_on_shutdown_) : WithContext(context_) , pool_size(pool_size_) + , flush_on_shutdown(flush_on_shutdown_) , queue_shards(pool_size) , pool(CurrentMetrics::AsynchronousInsertThreads, CurrentMetrics::AsynchronousInsertThreadsActive, pool_size) { @@ -143,8 +144,6 @@ AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t poo AsynchronousInsertQueue::~AsynchronousInsertQueue() { - /// TODO: add a setting for graceful shutdown. - LOG_TRACE(log, "Shutting down the asynchronous insertion queue"); shutdown = true; @@ -156,17 +155,18 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue() assert(dump_by_first_update_threads[i].joinable()); dump_by_first_update_threads[i].join(); + if (flush_on_shutdown) + { + for (auto & [_, elem] : shard.queue) + scheduleDataProcessingJob(elem.key, std::move(elem.data), getContext()); + } + else { - std::lock_guard lock(shard.mutex); for (auto & [_, elem] : shard.queue) - { for (const auto & entry : elem.data->entries) - { entry->finish(std::make_exception_ptr(Exception( ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout exceeded)"))); - } - } } } @@ -210,7 +210,9 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// to avoid buffering of huge amount of data in memory. auto read_buf = getReadBufferFromASTInsertQuery(query); - LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* trow_exception */ false, /* exact_limit */ {}); + LimitReadBuffer limit_buf( + *read_buf, settings.async_insert_max_data_size, + /*throw_exception=*/ false, /*exact_limit=*/ {}); WriteBufferFromString write_buf(bytes); copyData(limit_buf, write_buf); @@ -262,18 +264,19 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) assert(data); data->size_in_bytes += entry_data_size; - ++data->query_number; data->entries.emplace_back(entry); insert_future = entry->getFuture(); LOG_TRACE(log, "Have {} pending inserts with total {} bytes of data for query '{}'", data->entries.size(), data->size_in_bytes, key.query_str); + bool has_enough_bytes = data->size_in_bytes >= key.settings.async_insert_max_data_size; + bool has_enough_queries = data->entries.size() >= key.settings.async_insert_max_query_number && key.settings.async_insert_deduplicate; + /// Here we check whether we hit the limit on maximum data size in the buffer. /// And use setting from query context. /// It works, because queries with the same set of settings are already grouped together. - if (data->size_in_bytes >= key.settings.async_insert_max_data_size - || (data->query_number >= key.settings.async_insert_max_query_number && key.settings.async_insert_deduplicate)) + if (!flush_stopped && (has_enough_bytes || has_enough_queries)) { data_to_process = std::move(data); shard.iterators.erase(it); @@ -297,6 +300,47 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) }; } +void AsynchronousInsertQueue::flushAll() +{ + std::lock_guard flush_lock(flush_mutex); + + LOG_DEBUG(log, "Requested to flush asynchronous insert queue"); + + flush_stopped = true; + std::vector queues_to_flush(pool_size); + + for (size_t i = 0; i < pool_size; ++i) + { + std::lock_guard lock(queue_shards[i].mutex); + queues_to_flush[i] = std::move(queue_shards[i].queue); + queue_shards[i].iterators.clear(); + } + + size_t total_queries = 0; + size_t total_bytes = 0; + size_t total_entries = 0; + + for (auto & queue : queues_to_flush) + { + total_queries += queue.size(); + for (auto & [_, entry] : queue) + { + total_bytes += entry.data->size_in_bytes; + total_entries += entry.data->entries.size(); + scheduleDataProcessingJob(entry.key, std::move(entry.data), getContext()); + } + } + + LOG_DEBUG(log, + "Will wait for finishing of {} flushing jobs (about {} inserts, {} bytes, {} distinct queries)", + pool.active(), total_entries, total_bytes, total_queries); + + pool.wait(); + + LOG_DEBUG(log, "Finished flushing of asynchronous insert queue"); + flush_stopped = false; +} + void AsynchronousInsertQueue::processBatchDeadlines(size_t shard_num) { auto & shard = queue_shards[shard_num]; @@ -322,6 +366,9 @@ void AsynchronousInsertQueue::processBatchDeadlines(size_t shard_num) if (shutdown) return; + if (flush_stopped) + continue; + const auto now = std::chrono::steady_clock::now(); while (true) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 23a2860364d8..97294d70eadf 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -16,7 +16,7 @@ class AsynchronousInsertQueue : public WithContext public: using Milliseconds = std::chrono::milliseconds; - AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_); + AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_, bool flush_on_shutdown_); ~AsynchronousInsertQueue(); struct PushResult @@ -37,6 +37,7 @@ class AsynchronousInsertQueue : public WithContext std::unique_ptr insert_data_buffer; }; + void flushAll(); PushResult push(ASTPtr query, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } @@ -82,9 +83,7 @@ class AsynchronousInsertQueue : public WithContext using EntryPtr = std::shared_ptr; std::list entries; - size_t size_in_bytes = 0; - size_t query_number = 0; }; using InsertDataPtr = std::unique_ptr; @@ -112,6 +111,8 @@ class AsynchronousInsertQueue : public WithContext }; const size_t pool_size; + const bool flush_on_shutdown; + std::vector queue_shards; /// Logic and events behind queue are as follows: @@ -123,6 +124,10 @@ class AsynchronousInsertQueue : public WithContext /// (async_insert_max_data_size setting). If so, then again we dump the data. std::atomic shutdown{false}; + std::atomic flush_stopped{false}; + + /// A mutex that prevents concurrent forced flushes of queue. + mutable std::mutex flush_mutex; /// Dump the data only inside this pool. ThreadPool pool; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 36cb57c36781..f73429913b3a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -564,6 +565,17 @@ BlockIO InterpreterSystemQuery::execute() ); break; } + case Type::FLUSH_ASYNC_INSERT_QUEUE: + { + getContext()->checkAccess(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); + auto * queue = getContext()->getAsynchronousInsertQueue(); + if (!queue) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot flush asynchronous insert queue because it is not initialized"); + + queue->flushAll(); + break; + } case Type::STOP_LISTEN_QUERIES: case Type::START_LISTEN_QUERIES: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type); @@ -1156,6 +1168,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_FLUSH_LOGS); break; } + case Type::FLUSH_ASYNC_INSERT_QUEUE: + { + required_access.emplace_back(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); + break; + } case Type::RESTART_DISK: { required_access.emplace_back(AccessType::SYSTEM_RESTART_DISK); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index dfe2389edb7d..9e2dca8bb230 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -72,6 +72,7 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster START_REPLICATION_QUEUES, FLUSH_LOGS, FLUSH_DISTRIBUTED, + FLUSH_ASYNC_INSERT_QUEUE, STOP_DISTRIBUTED_SENDS, START_DISTRIBUTED_SENDS, START_THREAD_FUZZER, diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.reference b/tests/queries/0_stateless/02726_async_insert_flush_queue.reference new file mode 100644 index 000000000000..b94888d227ea --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.reference @@ -0,0 +1,5 @@ +JSONEachRow 3 +Values 2 +0 +0 +9 diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql new file mode 100644 index 000000000000..33f40eef14ec --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS t_async_inserts_flush; + +CREATE TABLE t_async_inserts_flush (a UInt64) ENGINE = Memory; + +SET async_insert = 1; +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_ms = 1000000; + +INSERT INTO t_async_inserts_flush VALUES (1) (2); +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 10} {"a": 20}; +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"} +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200} +INSERT INTO t_async_inserts_flush VALUES (3) (4) (5); + +SELECT sleep(1) FORMAT Null; + +SELECT format, length(entries.query_id) FROM system.asynchronous_inserts +WHERE database = currentDatabase() AND table = 't_async_inserts_flush' +ORDER BY format; + +SELECT count() FROM t_async_inserts_flush; + +SYSTEM FLUSH ASYNC INSERT QUEUE; + +SELECT count() FROM system.asynchronous_inserts; +SELECT count() FROM t_async_inserts_flush; + +DROP TABLE t_async_inserts_flush; From c9e30d3cf5f5f0ac9f35e2e08df429bacbe4cd25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:04:20 +0200 Subject: [PATCH 004/226] Properly check the limit for `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block` --- src/Core/Settings.h | 3 ++- src/Functions/sleep.h | 21 +++++++++++++++---- .../02725_sleep_max_time.reference | 0 .../0_stateless/02725_sleep_max_time.sql | 1 + 4 files changed, 20 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02725_sleep_max_time.reference create mode 100644 tests/queries/0_stateless/02725_sleep_max_time.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f1fe838b802..5aa054d43b06 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -633,7 +633,8 @@ class IColumn; M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ \ - M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function 'range' per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ + M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function `range` per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ + M(UInt64, function_sleep_max_microseconds_per_block, 3000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ 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) \ diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index d19608603081..93525c3f310b 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -9,7 +9,8 @@ #include #include #include -#include +#include + namespace ProfileEvents { @@ -40,11 +41,17 @@ enum class FunctionSleepVariant template class FunctionSleep : public IFunction { +private: + UInt64 max_microseconds; public: static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) + { + return std::make_shared>(context->getSettingsRef().function_sleep_max_microseconds_per_block); + } + + FunctionSleep(UInt64 max_microseconds_) : max_microseconds(max_microseconds_) { - return std::make_shared>(); } /// Get the name of the function. @@ -105,13 +112,19 @@ class FunctionSleep : public IFunction if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. - if (seconds > 3.0) /// The choice is arbitrary + if (seconds * 1e6 > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); if (!dry_run) { UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); UInt64 microseconds = static_cast(seconds * count * 1e6); + + if (microseconds > max_microseconds) + throw Exception(ErrorCodes::TOO_SLOW, + "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", + microseconds, size); + sleepForMicroseconds(microseconds); ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds); diff --git a/tests/queries/0_stateless/02725_sleep_max_time.reference b/tests/queries/0_stateless/02725_sleep_max_time.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/02725_sleep_max_time.sql b/tests/queries/0_stateless/02725_sleep_max_time.sql new file mode 100644 index 000000000000..b8378aee17eb --- /dev/null +++ b/tests/queries/0_stateless/02725_sleep_max_time.sql @@ -0,0 +1 @@ +SELECT * FROM system.numbers WHERE sleepEachRow(0.05) LIMIT 10; -- { serverError TOO_SLOW } From 3de0c319c2d6b6206196ece48b228f72f3a9aecd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:08:52 +0200 Subject: [PATCH 005/226] Add compatibility --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Functions/sleep.h | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5aa054d43b06..2ab4fe9b32a0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -634,7 +634,7 @@ class IColumn; M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ \ M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function `range` per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ - M(UInt64, function_sleep_max_microseconds_per_block, 3000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ + M(UInt64, function_sleep_max_microseconds_per_block, 3000000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ 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) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 266d14f645b1..33010dc6b3bc 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximim sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}}}, {"23.4", {{"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 93525c3f310b..db4f0e7dd3e8 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -112,7 +112,7 @@ class FunctionSleep : public IFunction if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. - if (seconds * 1e6 > max_microseconds) + if (max_microseconds && seconds * 1e6 > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); if (!dry_run) @@ -120,7 +120,7 @@ class FunctionSleep : public IFunction UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); UInt64 microseconds = static_cast(seconds * count * 1e6); - if (microseconds > max_microseconds) + if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", microseconds, size); From 582cf2ca8427c572a83c0bc249275c22fae6de5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:48:33 +0200 Subject: [PATCH 006/226] Update tests --- src/Functions/sleep.h | 6 +++--- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 1 + tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 4 ++-- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- .../queries/0_stateless/01192_rename_database_zookeeper.sh | 4 ++-- tests/queries/0_stateless/01238_http_memory_tracking.sh | 2 +- tests/queries/0_stateless/01246_buffer_flush.sql | 2 ++ tests/queries/0_stateless/01338_long_select_and_alter.sh | 2 +- .../0_stateless/01338_long_select_and_alter_zookeeper.sh | 2 +- .../01532_execute_merges_on_single_replica_long.sql | 1 + .../01715_background_checker_blather_zookeeper_long.sql | 1 + .../01737_clickhouse_server_wait_server_pool_long.sh | 2 +- 12 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index db4f0e7dd3e8..fba8293e5ffe 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -113,7 +113,7 @@ class FunctionSleep : public IFunction { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. if (max_microseconds && seconds * 1e6 > max_microseconds) - throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); + throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {}", max_microseconds, seconds); if (!dry_run) { @@ -122,8 +122,8 @@ class FunctionSleep : public IFunction if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, - "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", - microseconds, size); + "The maximum sleep time is {} microseconds. Requested: {} microseconds per block (of size {})", + max_microseconds, microseconds, size); sleepForMicroseconds(microseconds); ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index ccd9bbcf10ec..a31a71ce3812 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -65,6 +65,7 @@ echo 5 # run in background rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ + --function_sleep_max_microseconds_per_block 60 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e4dad56bc29d..e2a23258584d 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 $CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_dr $CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 4a3d35e48b71..634b19a76241 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -49,8 +49,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -nm -q " @@ -74,7 +74,7 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index dec1276111ad..ac516e83c84a 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT -q "SELECT engine, splitByChar('/', data_path)[-2], uuid, spl # 3. check RENAME don't wait for INSERT $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt (n UInt64) ENGINE=MergeTree ORDER BY n" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO default" 2>&1| grep -F "already exists" > /dev/null && echo "ok" @@ -60,7 +60,7 @@ $CLICKHOUSE_CLIENT -q "SELECT database, name, status, origin FROM system.diction $CLICKHOUSE_CLIENT -q "SELECT dictGet('test_01192_atomic.dict', '_part', toUInt64(1))" # 8. check RENAME don't wait for INSERT -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "RENAME DATABASE test_01192 TO test_01192_renamed" 2>&1| grep -F "not supported" > /dev/null && echo "ok" diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index 9b0fe8754161..eb42159ce15d 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -10,7 +10,7 @@ set -o pipefail # This is needed to keep at least one running query for user for the time of test. # (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) -${CLICKHOUSE_CLIENT} --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & +${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql index ac507d94b696..36bcaae383f7 100644 --- a/tests/queries/0_stateless/01246_buffer_flush.sql +++ b/tests/queries/0_stateless/01246_buffer_flush.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest +SET function_sleep_max_microseconds_per_block = 4000000; + drop table if exists data_01256; drop table if exists buffer_01256; diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh index 2e3080e9cfc1..04a10cfe55ee 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh index 12bc3b09472a..829352110f6a 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index f217b6094b24..d39ffdc4049c 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -44,6 +44,7 @@ SYSTEM STOP REPLICATION QUEUES execute_on_single_replica_r2; OPTIMIZE TABLE execute_on_single_replica_r1 FINAL SETTINGS replication_alter_partitions_sync=0; /* if we will check immediately we can find the log entry unchecked */ +SET function_sleep_max_microseconds_per_block = 4000000; SELECT * FROM numbers(4) where sleepEachRow(1); SELECT '****************************'; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index 87e1a0394880..32481be1bcd5 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -18,6 +18,7 @@ DETACH TABLE i20203_1; ATTACH TABLE i20203_2; -- sleep 10 seconds +SET function_sleep_max_microseconds_per_block = 10000000; SELECT number from numbers(10) where sleepEachRow(1) Format Null; SELECT num_tries < 50 diff --git a/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh b/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh index d83656e0e8cd..adab3906e5bf 100755 --- a/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh +++ b/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh @@ -54,7 +54,7 @@ if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null fi query_id="$CLICKHOUSE_DATABASE-$SECONDS" -$CLICKHOUSE_CLIENT_BINARY --query_id "$query_id" --host 127.1 --port "$server_port" --format Null -q 'select sleepEachRow(1) from numbers(10)' 2>/dev/null & +$CLICKHOUSE_CLIENT_BINARY --query_id "$query_id" --host 127.1 --port "$server_port" --format Null --function_sleep_max_microseconds_per_block 0 -q 'select sleepEachRow(1) from numbers(10)' 2>/dev/null & client_pid=$! # wait until the query will appear in processlist (max 10 second) From 95caa02cbc053f672ffa83a6dbe1a96259ea4d25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 23:28:08 +0200 Subject: [PATCH 007/226] Update test --- ...02494_zero_copy_and_projection_and_mutation_work_together.sql | 1 + .../02572_query_views_log_background_thread.reference | 1 + .../0_stateless/02572_query_views_log_background_thread.sql | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql index 7a51d86dd30f..b6ab9b7d0c3f 100644 --- a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql @@ -70,6 +70,7 @@ SYSTEM SYNC REPLICA wikistat2; -- it doesn't make test flaky, rarely we will not delete the parts because of cleanup thread was slow. -- Such condition will lead to successful queries. +SET function_sleep_max_microseconds_per_block = 5000000; SELECT 0 FROM numbers(5) WHERE sleepEachRow(1) = 1; select sum(hits), count() from wikistat1 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index eeba62c5dc87..22dfaf937811 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -4,6 +4,7 @@ insert into buffer_02572 values (1); select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait +SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; select * from data_02572; 1 diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql index dc229412b133..939c189c5fe2 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql @@ -22,6 +22,7 @@ insert into buffer_02572 values (1); select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait +SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; select * from data_02572; select * from copy_02572; From 748a21b791f5846a4f9f1d49d38fc077c7f9d3d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Apr 2023 01:44:03 +0200 Subject: [PATCH 008/226] Fix typo --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 33010dc6b3bc..e0d23d139f36 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximim sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, + {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}}}, {"23.4", {{"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, From 726222f1ea69018115642156a06c64ec546244d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:33:11 +0200 Subject: [PATCH 009/226] Fix tests --- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 2 +- tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 4 ++-- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- .../queries/0_stateless/01192_rename_database_zookeeper.sh | 4 ++-- tests/queries/0_stateless/01238_http_memory_tracking.sh | 2 +- tests/queries/0_stateless/01338_long_select_and_alter.sh | 2 +- .../0_stateless/01338_long_select_and_alter_zookeeper.sh | 2 +- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index a31a71ce3812..926557e4ba60 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -65,7 +65,7 @@ echo 5 # run in background rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ - --function_sleep_max_microseconds_per_block 60 \ + --function_sleep_max_microseconds_per_block 60000000 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e2a23258584d..bcaa70abbb55 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 $CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_dr $CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 634b19a76241..decbe136fc4c 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -49,8 +49,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -nm -q " @@ -74,7 +74,7 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index ac516e83c84a..6dd7ff3cdc85 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT -q "SELECT engine, splitByChar('/', data_path)[-2], uuid, spl # 3. check RENAME don't wait for INSERT $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt (n UInt64) ENGINE=MergeTree ORDER BY n" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15000000 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO default" 2>&1| grep -F "already exists" > /dev/null && echo "ok" @@ -60,7 +60,7 @@ $CLICKHOUSE_CLIENT -q "SELECT database, name, status, origin FROM system.diction $CLICKHOUSE_CLIENT -q "SELECT dictGet('test_01192_atomic.dict', '_part', toUInt64(1))" # 8. check RENAME don't wait for INSERT -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "RENAME DATABASE test_01192 TO test_01192_renamed" 2>&1| grep -F "not supported" > /dev/null && echo "ok" diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index eb42159ce15d..26d3dd8acd4c 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -10,7 +10,7 @@ set -o pipefail # This is needed to keep at least one running query for user for the time of test. # (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) -${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & +${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5000000 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh index 04a10cfe55ee..fcdfa2dec82a 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh index 829352110f6a..50ade3fad45f 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 From e159ee84e918c587f873a27665ca346cb3b4f7db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:38:30 +0200 Subject: [PATCH 010/226] Fix tests --- .../0_stateless/01098_temporary_and_external_tables.sh | 2 +- .../01532_execute_merges_on_single_replica_long.sql | 2 +- tests/queries/0_stateless/02473_optimize_old_parts.sh | 2 +- tests/queries/0_stateless/02530_dictionaries_update_field.sh | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh index 860529a26e54..9ed78fd9f816 100755 --- a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh +++ b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh @@ -25,7 +25,7 @@ echo "SELECT COUNT() FROM $internal_table_name" | ${CLICKHOUSE_CURL} -m 60 -sSgk echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&file_format=CSV&file_types=UInt64&query=SELECT+sum((number+GLOBAL+IN+(SELECT+number+AS+n+FROM+remote('127.0.0.2',+numbers(5))+WHERE+n+GLOBAL+IN+(SELECT+*+FROM+tmp_table)+AND+n+GLOBAL+NOT+IN+(SELECT+*+FROM+file)+))+AS+res),+sum(number*res)+FROM+remote('127.0.0.2',+numbers(10))" -echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&file_format=CSV&file_types=UInt64&query=SELECT+_1%2BsleepEachRow(3)+FROM+file" & +echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&function_sleep_max_microseconds_per_block=0&file_format=CSV&file_types=UInt64&query=SELECT+_1%2BsleepEachRow(3)+FROM+file" & wait ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE tmp_table" diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index 4bd5e79d1b3e..30beb29251e9 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -44,7 +44,7 @@ SYSTEM STOP REPLICATION QUEUES execute_on_single_replica_r2; OPTIMIZE TABLE execute_on_single_replica_r1 FINAL SETTINGS replication_alter_partitions_sync=0; /* if we will check immediately we can find the log entry unchecked */ -SET function_sleep_max_microseconds_per_block = 4000000; +SET function_sleep_max_microseconds_per_block = 10000000; SELECT * FROM numbers(4) where sleepEachRow(1); SELECT '****************************'; diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sh b/tests/queries/0_stateless/02473_optimize_old_parts.sh index 0c2dd04d0242..b563bc31b39a 100755 --- a/tests/queries/0_stateless/02473_optimize_old_parts.sh +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sh @@ -61,7 +61,7 @@ INSERT INTO test_with_merge SELECT 3;" wait_for_number_of_parts 'test_with_merge' 1 100 $CLICKHOUSE_CLIENT -nmq " -SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one +SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one SELECT (now() - modification_time) > 5 FROM system.parts WHERE database = currentDatabase() AND table='test_with_merge' AND active; DROP TABLE test_with_merge;" diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 569466fe6063..44000e5d2cd7 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -53,13 +53,13 @@ for layout in "${layouts[@]}"; do SELECT key, value FROM $dictionary_name ORDER BY key ASC; INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); - SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; -- { echoOff } From fbda7974a5424b79a952fa30b16b7cd3c390bdc8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:39:23 +0200 Subject: [PATCH 011/226] Fix tests --- .../queries/0_stateless/02676_optimize_old_parts_replicated.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh b/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh index 2202a349c569..c1f28f9f0794 100755 --- a/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh +++ b/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh @@ -61,7 +61,7 @@ INSERT INTO test_replicated SELECT 3;" wait_for_number_of_parts 'test_replicated' 1 100 $CLICKHOUSE_CLIENT -nmq " -SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one +SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one SELECT (now() - modification_time) > 5 FROM system.parts WHERE database = currentDatabase() AND table='test_replicated' AND active; DROP TABLE test_replicated;" From 08a9d97de74a27bd28d7cc387d7f5cdba707d6cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:40:43 +0200 Subject: [PATCH 012/226] Fix tests --- tests/queries/0_stateless/02352_rwlock.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh index 7a0b9ef8911f..7505a03a3825 100755 --- a/tests/queries/0_stateless/02352_rwlock.sh +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -51,7 +51,7 @@ while :; do insert_query_id="insert-$(random_str 10)" # 20 seconds sleep - $CLICKHOUSE_CLIENT --query_id "$insert_query_id" -q "INSERT INTO ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SELECT sleepEachRow(1) FROM numbers(20) GROUP BY number" & + $CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 20000000 --query_id "$insert_query_id" -q "INSERT INTO ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SELECT sleepEachRow(1) FROM numbers(20) GROUP BY number" & if ! wait_query_by_id_started "$insert_query_id"; then wait continue From 7c03801bf7da6803e47f57ab78478c33a9c9a764 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:54:57 +0200 Subject: [PATCH 013/226] Update a test --- tests/queries/0_stateless/02725_parquet_preserve_order.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_parquet_preserve_order.sh b/tests/queries/0_stateless/02725_parquet_preserve_order.sh index ea3e4219e353..ac29ef3f3615 100755 --- a/tests/queries/0_stateless/02725_parquet_preserve_order.sh +++ b/tests/queries/0_stateless/02725_parquet_preserve_order.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # It'll be read into two blocks. The first block will sleep 2x longer than the second. # So reordering is very likely if the order-preservation doesn't work. -$CLICKHOUSE_LOCAL -q "select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1" +$CLICKHOUSE_LOCAL -q "select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, function_sleep_max_microseconds_per_block = 6000000" -$CLICKHOUSE_LOCAL -q "explain pipeline select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2" -$CLICKHOUSE_LOCAL -q "explain pipeline select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2" From 63b559df17a07e42768c4425538426e245d829fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 06:49:41 +0200 Subject: [PATCH 014/226] Update a test --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 40f2c0ee4005..88c910e03133 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 1eb939766bc78a59dd11b3534f4fd7b693d75e21 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 16 May 2023 17:48:49 +0000 Subject: [PATCH 015/226] add test --- src/Interpreters/AsynchronousInsertQueue.cpp | 4 + src/Interpreters/AsynchronousInsertQueue.h | 1 + .../02726_async_insert_flush_stress.reference | 1 + .../02726_async_insert_flush_stress.sh | 86 +++++++++++++++++++ 4 files changed, 92 insertions(+) create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_stress.reference create mode 100755 tests/queries/0_stateless/02726_async_insert_flush_stress.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 4592e92151ee..e176c7afd76f 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -318,6 +318,7 @@ void AsynchronousInsertQueue::flushAll() LOG_DEBUG(log, "Requested to flush asynchronous insert queue"); + /// Disable background flushes to avoid adding new elements to the queue. flush_stopped = true; std::vector queues_to_flush(pool_size); @@ -343,10 +344,13 @@ void AsynchronousInsertQueue::flushAll() } } + /// Note that jobs scheduled before the call of 'flushAll' are not counted here. LOG_DEBUG(log, "Will wait for finishing of {} flushing jobs (about {} inserts, {} bytes, {} distinct queries)", pool.active(), total_entries, total_bytes, total_queries); + /// Wait until all jobs are finished. That includes also jobs + /// that were scheduled before the call of 'flushAll'. pool.wait(); LOG_DEBUG(log, "Finished flushing of asynchronous insert queue"); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 455e486c7981..b22b0c739077 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -38,6 +38,7 @@ class AsynchronousInsertQueue : public WithContext std::unique_ptr insert_data_buffer; }; + /// Force flush the whole queue. void flushAll(); PushResult push(ASTPtr query, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.reference b/tests/queries/0_stateless/02726_async_insert_flush_stress.reference new file mode 100644 index 000000000000..573541ac9702 --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh new file mode 100755 index 000000000000..4685e49b96db --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -0,0 +1,86 @@ +#!/usr/bin/env bash +# Tags: long + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +function insert1() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV +1,"a" +2,"b" +' + done +} + +function insert2() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' + done +} + +function insert3() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" + done +} + +function select1() +{ + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" + done +} + +function select2() +{ + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" + done +} + +function flush1() +{ + while true; do + sleep 0.2 + ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" + done +} + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id" + +TIMEOUT=10 + +export -f insert1 +export -f insert2 +export -f insert3 +export -f select1 +export -f select2 +export -f flush1 + +for _ in {1..5}; do + timeout $TIMEOUT bash -c insert1 & + timeout $TIMEOUT bash -c insert2 & + timeout $TIMEOUT bash -c insert3 & +done + +timeout $TIMEOUT bash -c select1 & +timeout $TIMEOUT bash -c select2 & +timeout $TIMEOUT bash -c flush1 & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"; From 05a90a2e971ae7538ed72e1a3db02523c91b67d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 17 May 2023 12:19:00 +0000 Subject: [PATCH 016/226] fix tests --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../0_stateless/02117_show_create_table_system.reference | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index ec245d8b9e02..eb8b912f03bd 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -133,6 +133,7 @@ SYSTEM SYNC TRANSACTION LOG ['SYNC TRANSACTION LOG'] GLOBAL SYSTEM SYSTEM SYNC FILE CACHE ['SYNC FILE CACHE'] GLOBAL SYSTEM SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH +SYSTEM FLUSH ASYNC INSERT QUEUE ['FLUSH ASYNC INSERT QUEUE'] GLOBAL SYSTEM FLUSH SYSTEM FLUSH [] \N SYSTEM SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','START THREAD FUZZER','STOP THREAD FUZZER'] GLOBAL SYSTEM SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM 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 09cc62dac003..85cdc278892d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -581,10 +581,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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163)) + `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 ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 616904cd790473ca8075a8175a6334dd837b5bca Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 23 May 2023 15:50:52 +0000 Subject: [PATCH 017/226] Add encryptConfig() --- src/Common/Config/ConfigProcessor.cpp | 34 +++++++++++++++++++++++++++ src/Common/Config/ConfigProcessor.h | 5 ++++ src/Common/Config/ConfigReloader.cpp | 1 + src/Daemon/BaseDaemon.cpp | 1 + 4 files changed, 41 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5bbc8eae0de1..76e4ea1ebd1a 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -171,6 +171,33 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } +void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) +{ + for (Node * node = config_root->firstChild(); node;) + { + if (node->nodeType() == Node::ELEMENT_NODE) + { + // NamedNodeMapPtr attributes = node->attributes(); + Element & element = dynamic_cast(*node); + if (element.hasAttribute("enc_codec")) + { + LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); + // for (Node * child_node = node->firstChild(); child_node;) + // { + // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); + // child_node = child_node->nextSibling(); + // } + Node * child_node = node->firstChild(); + child_node->setNodeValue("encrypted_" + child_node->getNodeValue() + "_encrypted"); + } + } + + encryptRecursive(node); + + node = node->nextSibling(); + } +} + void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, const Node * with_root) { const NodeListPtr with_nodes = with_root->childNodes(); @@ -700,6 +727,13 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } +void ConfigProcessor::encryptConfig(LoadedConfig & loaded_config) +{ + Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); + encryptRecursive(config_root); + loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); +} + void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) { try diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0ca3e46db884..2f0046bc39c8 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,6 +92,9 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); + /// Encrypt nodes in config with specified encryption attributes + void encryptConfig(LoadedConfig & loaded_config); + /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir); @@ -124,6 +127,8 @@ class ConfigProcessor using NodePtr = Poco::AutoPtr; + void encryptRecursive(Poco::XML::Node * config_root); + void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); void merge(XMLDocumentPtr config, XMLDocumentPtr with); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index de7011b67bfa..896bd5949d99 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -130,6 +130,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); + config_processor.encryptConfig(loaded_config); /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 4780dfed4b21..2634439ee14b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -663,6 +663,7 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); + DB::ConfigProcessor(config_path).encryptConfig(loaded_config); /// Write core dump on crash. { From dd78008c9ec586a213e0e541b70dfe5055f7df0e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 25 May 2023 09:36:41 +0000 Subject: [PATCH 018/226] Rename encryptConfig() into decryptConfig() --- src/Common/Config/ConfigProcessor.cpp | 12 ++++++------ src/Common/Config/ConfigProcessor.h | 6 +++--- src/Common/Config/ConfigReloader.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 2 +- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 76e4ea1ebd1a..3f9535205d82 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -171,7 +171,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } -void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) +void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { for (Node * node = config_root->firstChild(); node;) { @@ -179,7 +179,7 @@ void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) { // NamedNodeMapPtr attributes = node->attributes(); Element & element = dynamic_cast(*node); - if (element.hasAttribute("enc_codec")) + if (element.hasAttribute("encryption_codec")) { LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); // for (Node * child_node = node->firstChild(); child_node;) @@ -188,11 +188,11 @@ void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) // child_node = child_node->nextSibling(); // } Node * child_node = node->firstChild(); - child_node->setNodeValue("encrypted_" + child_node->getNodeValue() + "_encrypted"); + child_node->setNodeValue("decrypted_" + child_node->getNodeValue() + "_decrypted"); } } - encryptRecursive(node); + decryptRecursive(node); node = node->nextSibling(); } @@ -727,10 +727,10 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } -void ConfigProcessor::encryptConfig(LoadedConfig & loaded_config) +void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); - encryptRecursive(config_root); + decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 2f0046bc39c8..bc2f923f7054 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,8 +92,8 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// Encrypt nodes in config with specified encryption attributes - void encryptConfig(LoadedConfig & loaded_config); + /// Decrypt nodes in config with specified encryption attributes + void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ @@ -127,7 +127,7 @@ class ConfigProcessor using NodePtr = Poco::AutoPtr; - void encryptRecursive(Poco::XML::Node * config_root); + void decryptRecursive(Poco::XML::Node * config_root); void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 896bd5949d99..a4d2cb3d3053 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -130,7 +130,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); - config_processor.encryptConfig(loaded_config); + config_processor.decryptConfig(loaded_config); /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 2634439ee14b..4b1cd4e036ea 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -663,7 +663,7 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); - DB::ConfigProcessor(config_path).encryptConfig(loaded_config); + DB::ConfigProcessor(config_path).decryptConfig(loaded_config); /// Write core dump on crash. { From 5f73681b00fb1a13873c9a8e6b07c7f57c335668 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 25 May 2023 15:51:20 +0000 Subject: [PATCH 019/226] Make working note descryption --- src/Common/Config/ConfigProcessor.cpp | 44 +++++++++++++++++++++++++-- src/Common/Config/ConfigProcessor.h | 2 +- utils/config-processor/CMakeLists.txt | 3 +- 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 3f9535205d82..fdfc63438768 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -26,6 +26,10 @@ #include #include #include +#include +#include +#include +#include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -181,14 +185,47 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { - LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); + LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); // for (Node * child_node = node->firstChild(); child_node;) // { // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); // child_node = child_node->nextSibling(); // } - Node * child_node = node->firstChild(); - child_node->setNodeValue("decrypted_" + child_node->getNodeValue() + "_decrypted"); + + Node * text_node = node->firstChild(); + auto codec_128 = DB::CompressionCodecEncrypted(DB::AES_128_GCM_SIV); + // DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, ""); + + /* + DB::Memory<> memory1; + std::string password="abcd"; + memory1.resize(password.size() + codec_128.getAdditionalSizeAtTheEndOfBuffer() + codec_128.getHeaderSize()+100); + auto bytes_written = codec_128.compress(password.data(), static_cast(password.size()), memory1.data()); + // std::string encrypted_password = std::string(memory1.data(), memory1.size()); + std::string encrypted_password = std::string(memory1.data(), bytes_written); + std::string password_hex; + boost::algorithm::hex(encrypted_password.begin(), encrypted_password.end(), std::back_inserter(password_hex)); + LOG_DEBUG(log, "Encrypted password: '{}'.", password_hex); + */ + + DB::Memory<> memory; + std::string encrypted_value; + + try + { + boost::algorithm::unhex(text_node->getNodeValue(), std::back_inserter(encrypted_value)); + // boost::algorithm::unhex(password_hex, std::back_inserter(encrypted_value)); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text for {}, check for valid characters [0-9a-fA-F] and length", node->nodeName()); + } + + memory.resize(codec_128.readDecompressedBlockSize(encrypted_value.data()) + codec_128.getAdditionalSizeAtTheEndOfBuffer()); + codec_128.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + std::string decrypted_value = std::string(memory.data(), memory.size()); + LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); + text_node->setNodeValue(decrypted_value); } } @@ -729,6 +766,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { + DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index bc2f923f7054..479a0053efac 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,7 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// Decrypt nodes in config with specified encryption attributes + /// crypt nodes in config with specified encryption attributes void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 53b6163ba871..00cbfbba659a 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,3 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) +target_link_libraries(config-processor PRIVATE dbms clickhouse_common_config_no_zookeeper_log) +target_link_libraries(config-processor PUBLIC clickhouse_parsers clickhouse_common_io common ch_contrib::lz4) From cd8eb44f0c54945f4777ed3e50e08b057ee41f43 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 31 May 2023 14:03:11 +0000 Subject: [PATCH 020/226] Add encryptValue(), decryptValue() and exceptions --- src/Common/Config/ConfigProcessor.cpp | 94 ++++++++++++++++----------- src/Common/Config/ConfigProcessor.h | 8 ++- 2 files changed, 63 insertions(+), 39 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index fdfc63438768..b6db53018f49 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -46,6 +46,17 @@ namespace ErrorCodes extern const int CANNOT_LOAD_CONFIG; } +/// Get method for string name. Throw exception for wrong name +EncryptionMethod getEncryptionMethod(const std::string & name) +{ + if (name == "AES_128_GCM_SIV") + return AES_128_GCM_SIV; + else if (name == "AES_256_GCM_SIV") + return AES_256_GCM_SIV; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); +} + /// For cutting preprocessed path to this base static std::string main_config_path; @@ -175,57 +186,64 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) +{ + auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + + DB::Memory<> memory1; + memory1.resize(value.size() + codec.getAdditionalSizeAtTheEndOfBuffer() + codec.getHeaderSize()+100); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory1.data()); + std::string encrypted_value = std::string(memory1.data(), bytes_written); + std::string hex_value; + boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); + LOG_DEBUG(log, "Encrypted value: '{}'.", hex_value); + return hex_value; +} + +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) +{ + auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + + DB::Memory<> memory; + std::string encrypted_value; + + try + { + boost::algorithm::unhex(value, std::back_inserter(encrypted_value)); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text, check for valid characters [0-9a-fA-F] and length"); + } + + memory.resize(codec.readDecompressedBlockSize(encrypted_value.data()) + codec.getAdditionalSizeAtTheEndOfBuffer()); + codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + std::string decrypted_value = std::string(memory.data(), memory.size()); + LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); + return decrypted_value; +} + void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { for (Node * node = config_root->firstChild(); node;) { if (node->nodeType() == Node::ELEMENT_NODE) { - // NamedNodeMapPtr attributes = node->attributes(); Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); - // for (Node * child_node = node->firstChild(); child_node;) - // { - // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); - // child_node = child_node->nextSibling(); - // } + + const NodeListPtr children = element.childNodes(); + if (children->length() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); Node * text_node = node->firstChild(); - auto codec_128 = DB::CompressionCodecEncrypted(DB::AES_128_GCM_SIV); - // DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, ""); - - /* - DB::Memory<> memory1; - std::string password="abcd"; - memory1.resize(password.size() + codec_128.getAdditionalSizeAtTheEndOfBuffer() + codec_128.getHeaderSize()+100); - auto bytes_written = codec_128.compress(password.data(), static_cast(password.size()), memory1.data()); - // std::string encrypted_password = std::string(memory1.data(), memory1.size()); - std::string encrypted_password = std::string(memory1.data(), bytes_written); - std::string password_hex; - boost::algorithm::hex(encrypted_password.begin(), encrypted_password.end(), std::back_inserter(password_hex)); - LOG_DEBUG(log, "Encrypted password: '{}'.", password_hex); - */ - - DB::Memory<> memory; - std::string encrypted_value; - - try - { - boost::algorithm::unhex(text_node->getNodeValue(), std::back_inserter(encrypted_value)); - // boost::algorithm::unhex(password_hex, std::back_inserter(encrypted_value)); - } - catch (const std::exception &) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text for {}, check for valid characters [0-9a-fA-F] and length", node->nodeName()); - } + if (text_node->nodeType() != Node::TEXT_NODE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have text node", node->nodeName()); - memory.resize(codec_128.readDecompressedBlockSize(encrypted_value.data()) + codec_128.getAdditionalSizeAtTheEndOfBuffer()); - codec_128.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); - std::string decrypted_value = std::string(memory.data(), memory.size()); - LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); - text_node->setNodeValue(decrypted_value); + auto encryption_codec = element.getAttribute("encryption_codec"); + text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); } } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 479a0053efac..c9b227863f06 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,13 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// crypt nodes in config with specified encryption attributes + /// Encrypt text value + std::string encryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt value + std::string decryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt nodes in config with specified encryption attributes void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. From fd8c5992889728c76d231a4f96c577bc6578017d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 31 May 2023 15:16:18 +0000 Subject: [PATCH 021/226] Add encrypt_decrypt example --- src/Common/examples/CMakeLists.txt | 3 ++ src/Common/examples/encrypt_decrypt.cpp | 50 +++++++++++++++++++++++++ utils/config-processor/CMakeLists.txt | 3 +- 3 files changed, 54 insertions(+), 2 deletions(-) create mode 100644 src/Common/examples/encrypt_decrypt.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index d095ab3a1bee..12a2b59ff775 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -82,3 +82,6 @@ endif() clickhouse_add_executable (interval_tree interval_tree.cpp) target_link_libraries (interval_tree PRIVATE dbms) + +clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) +target_link_libraries (encrypt_decrypt PRIVATE dbms) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp new file mode 100644 index 000000000000..cd48963c47a4 --- /dev/null +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include + + +int main(int argc, char ** argv) +{ + try + { + if (argc != 5) + { + std::cerr << "usage: " << argv[0] << " path action codec value" << std::endl; + return 3; + } + + std::string action = argv[2]; + std::string codec_name = argv[3]; + std::string value = argv[4]; + DB::ConfigProcessor processor(argv[1], false, true); + + auto loaded_config = processor.loadConfig(); + + DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + + if (action == "-e") + std::cout << processor.encryptValue(codec_name, value) << std::endl; + else if (action == "-d") + std::cout << processor.decryptValue(codec_name, value) << std::endl; + else + std::cerr << "Unknown action: " << action << std::endl; + } + catch (Poco::Exception & e) + { + std::cerr << "Exception: " << e.displayText() << std::endl; + return 1; + } + catch (std::exception & e) + { + std::cerr << "std::exception: " << e.what() << std::endl; + return 3; + } + catch (...) + { + std::cerr << "Some exception" << std::endl; + return 2; + } + + return 0; +} diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 00cbfbba659a..80c3535ef4eb 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,3 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE dbms clickhouse_common_config_no_zookeeper_log) -target_link_libraries(config-processor PUBLIC clickhouse_parsers clickhouse_common_io common ch_contrib::lz4) +target_link_libraries(config-processor PRIVATE dbms) From 0708caeb770e88a4805e084eeb01465c85fa45e2 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 09:01:01 +0000 Subject: [PATCH 022/226] Fix style --- src/Common/Config/ConfigProcessor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index b6db53018f49..055a497fb383 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int CANNOT_LOAD_CONFIG; + extern const int BAD_ARGUMENTS; } /// Get method for string name. Throw exception for wrong name From 2ccec017717e57b0eb1bdfb573f6f09e5201446d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 09:53:31 +0000 Subject: [PATCH 023/226] Set correct memory size for encrypt/decrypt --- src/Common/Config/ConfigProcessor.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 055a497fb383..99bea019c3b7 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -191,13 +191,12 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const { auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); - DB::Memory<> memory1; - memory1.resize(value.size() + codec.getAdditionalSizeAtTheEndOfBuffer() + codec.getHeaderSize()+100); - auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory1.data()); - std::string encrypted_value = std::string(memory1.data(), bytes_written); + DB::Memory<> memory; + memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); + std::string encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); - LOG_DEBUG(log, "Encrypted value: '{}'.", hex_value); return hex_value; } @@ -217,10 +216,9 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text, check for valid characters [0-9a-fA-F] and length"); } - memory.resize(codec.readDecompressedBlockSize(encrypted_value.data()) + codec.getAdditionalSizeAtTheEndOfBuffer()); + memory.resize(codec.readDecompressedBlockSize(encrypted_value.data())); codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); std::string decrypted_value = std::string(memory.data(), memory.size()); - LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); return decrypted_value; } From d5add614daa2e6f7f0a18eaada22f5c43a057934 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 12:48:45 +0000 Subject: [PATCH 024/226] Add text memo for encrypt_decrypt --- src/Common/examples/encrypt_decrypt.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index cd48963c47a4..542e173deb92 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -3,6 +3,12 @@ #include #include +/** This test program encrypts or decrypts text values using AES_128_GCM_SIV or AES_256_GCM_SIV codecs. + * Keys for codecs are loaded from section of configuration file. + * + * How to use: + * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test + */ int main(int argc, char ** argv) { @@ -10,17 +16,22 @@ int main(int argc, char ** argv) { if (argc != 5) { - std::cerr << "usage: " << argv[0] << " path action codec value" << std::endl; + std::cerr << "Usage:" << std::endl + << " " << argv[0] << " path action codec value" << std::endl + << "path: path to configuration file." << std::endl + << "action: -e for encryption and -d for decryption." << std::endl + << "codec: AES_128_GCM_SIV or AES_256_GCM_SIV." << std::endl << std::endl + << "Example:" << std::endl + << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test"; return 3; } std::string action = argv[2]; std::string codec_name = argv[3]; std::string value = argv[4]; - DB::ConfigProcessor processor(argv[1], false, true); + DB::ConfigProcessor processor(argv[1], false, true); auto loaded_config = processor.loadConfig(); - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); if (action == "-e") From e269235dbcf32f7e507370e1bff74a202a33446c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 13:09:21 +0000 Subject: [PATCH 025/226] Make decryptRecursive() go through element nodes only --- src/Common/Config/ConfigProcessor.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 99bea019c3b7..df25a9a3825a 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -231,8 +231,6 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { - LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); - const NodeListPtr children = element.childNodes(); if (children->length() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); @@ -244,10 +242,8 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) auto encryption_codec = element.getAttribute("encryption_codec"); text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); } + decryptRecursive(node); } - - decryptRecursive(node); - node = node->nextSibling(); } } From 88bf4e49d4767b6a3a3ccfcc383a42ca90ae12f1 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 17:40:40 -0400 Subject: [PATCH 026/226] update for min_chunk_bytes_for_parallel_parsing --- docs/en/sql-reference/transactions.md | 53 +++++++++++++++++++-------- 1 file changed, 37 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index 68fbfe0b22a8..1ca2db44b130 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -3,23 +3,44 @@ slug: /en/guides/developer/transactional --- # Transactional (ACID) support -INSERT into one partition* in one table* of MergeTree* family up to max_insert_block_size rows* is transactional (ACID): -- Atomic: INSERT is succeeded or rejected as a whole: if confirmation is sent to the client, all rows INSERTed; if error is sent to the client, no rows INSERTed. +## Case 1: INSERT into one partition, of one table, of the MergeTree* family + +This is transactional (ACID) if the number of rows inserted is less than or equal to `max_insert_block_size rows`, and in the case of data in TSV, TKSV, CSV, or JSONEachRow format if the number of bytes is less than `min_chunk_bytes_for_parallel_parsing`: +- Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. -- Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as if before INSERT or after successful INSERT; no partial state is seen; -- Durable: successful INSERT is written to the filesystem before answering to the client, on single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). -* If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own; -* INSERT into multiple tables with one statement is possible if materialized views are involved; -* INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional; -* another example: insert into Buffer tables is neither atomic nor isolated or consistent or durable; -* atomicity is ensured even if `async_insert` is enabled, but it can be turned off by the wait_for_async_insert setting; -* max_insert_block_size is 1 000 000 by default and can be adjusted as needed; -* if client did not receive the answer from the server, the client does not know if transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties; -* ClickHouse is using MVCC with snapshot isolation internally; -* all ACID properties are valid even in case of server kill / crash; -* either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in typical setup; -* "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) -* this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. +- Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen +- Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). +- INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). + +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family + +Same as Case 1 above, with this detail: +- If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own + + +## Case 3: INSERT into one distributed table of the MergeTree* family + +Same as Case 1 above, with this detail: +- INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional + +## Case 4: Using a Buffer table + +- insert into Buffer tables is neither atomic nor isolated nor consistent nor durable + +## Case 5: Using async_insert + +Same as Case 1 above, with this detail: +- atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. + +## Notes +- `max_insert_block_size` is 1 000 000 by default and can be adjusted as needed +- `min_chunk_bytes_for_parallel_parsing` is 1 000 000 by default and can be adjusted as needed +- if the client did not receive an answer from the server, the client does not know if the transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties +- ClickHouse is using MVCC with snapshot isolation internally +- all ACID properties are valid even in the case of server kill/crash +- either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in the typical setup +- "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) +- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback). ## Transactions, Commit, and Rollback From d868e35863c3a80c9924b347ac017e9e93c33ba2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 19:08:44 -0400 Subject: [PATCH 027/226] update spelling list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a90..0787ead76cf5 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -169,6 +169,7 @@ SelfManaged Stateful Submodules Subqueries +TKSV TSVRaw TSan TabItem From d316add2f1f6ffa9cf6f2a1107a4d7d69960c72a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 6 Jun 2023 15:24:29 +0000 Subject: [PATCH 028/226] Add integration test test_config_decryption --- .../test_config_decryption/__init__.py | 0 .../test_config_decryption/configs/config.xml | 12 +++++++ .../test_config_decryption/test.py | 31 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/integration/test_config_decryption/__init__.py create mode 100644 tests/integration/test_config_decryption/configs/config.xml create mode 100644 tests/integration/test_config_decryption/test.py diff --git a/tests/integration/test_config_decryption/__init__.py b/tests/integration/test_config_decryption/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_config_decryption/configs/config.xml b/tests/integration/test_config_decryption/configs/config.xml new file mode 100644 index 000000000000..5c274128e39d --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test.py b/tests/integration/test_config_decryption/test.py new file mode 100644 index 000000000000..a3cb1bb57f37 --- /dev/null +++ b/tests/integration/test_config_decryption/test.py @@ -0,0 +1,31 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/config.xml"]) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_successful_decryption(started_cluster): + assert ( + node.query( + "select value from system.server_settings where name ='max_table_size_to_drop'" + ) + == "60000000000\n" + ) + assert ( + node.query( + "select value from system.server_settings where name ='max_partition_size_to_drop'" + ) + == "40000000000\n" + ) From 9cd0d5e6db0ce68ffdc320e2f73d17531fb54ec0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Jun 2023 11:49:41 +0000 Subject: [PATCH 029/226] move settings to server_settings --- programs/server/Server.cpp | 9 +++------ src/Core/ServerSettings.h | 2 ++ 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a64676cfa01b..5496720e5dc5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1438,15 +1438,12 @@ try /// This is needed to load proper values of background_pool_size etc. global_context->initializeBackgroundExecutorsIfNeeded(); - size_t async_insert_threads = config().getUInt("async_insert_threads", 16); - bool async_insert_queue_flush_on_shutdown = config().getBool("async_insert_queue_flush_on_shutdown", false); - - if (async_insert_threads) + if (server_settings.async_insert_threads) { global_context->setAsynchronousInsertQueue(std::make_shared( global_context, - async_insert_threads, - async_insert_queue_flush_on_shutdown)); + server_settings.async_insert_threads, + server_settings.async_insert_queue_flush_on_shutdown)); } size_t mark_cache_size = server_settings.mark_cache_size; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1a9f226041b6..ca27cbdbf19d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -48,6 +48,8 @@ namespace DB M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Limit on total memory usage for merges and mutations. Zero means Unlimited.", 0) \ M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to ram ratio. Allows to lower memory limit on low-memory systems.", 0) \ M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ + M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ + M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ \ M(UInt64, max_concurrent_queries, 0, "Limit on total number of concurrently executed queries. Zero means Unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Limit on total number of concurrently insert queries. Zero means Unlimited.", 0) \ From ff1b069cdb9301f5c13bed69d70c484210693dea Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Jun 2023 21:15:35 +0000 Subject: [PATCH 030/226] fix test --- tests/queries/0_stateless/02726_async_insert_flush_queue.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index 33f40eef14ec..98e78045b85e 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP TABLE IF EXISTS t_async_inserts_flush; CREATE TABLE t_async_inserts_flush (a UInt64) ENGINE = Memory; From 60499164b3acc8663a10836233926c0f997ac381 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Jun 2023 14:35:11 +0000 Subject: [PATCH 031/226] fix tests --- tests/queries/0_stateless/02726_async_insert_flush_stress.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 4685e49b96db..5fafb773d164 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-parallel set -e From 1bce32c1cc1d9e2b0aeea93c01947646e18c52b3 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 9 Jun 2023 15:18:46 +0000 Subject: [PATCH 032/226] Add tests for wrong settings --- .../configs/config_invalid_chars.xml | 12 +++++++ .../configs/config_no_encryption_codecs.xml | 4 +++ .../configs/config_subnodes.xml | 10 ++++++ .../configs/config_wrong_method.xml | 12 +++++++ .../test_wrong_settings.py | 34 +++++++++++++++++++ 5 files changed, 72 insertions(+) create mode 100644 tests/integration/test_config_decryption/configs/config_invalid_chars.xml create mode 100644 tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml create mode 100644 tests/integration/test_config_decryption/configs/config_subnodes.xml create mode 100644 tests/integration/test_config_decryption/configs/config_wrong_method.xml create mode 100644 tests/integration/test_config_decryption/test_wrong_settings.py diff --git a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml new file mode 100644 index 000000000000..49bf51b5bad5 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml new file mode 100644 index 000000000000..07bf69d17c8b --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml @@ -0,0 +1,4 @@ + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_subnodes.xml b/tests/integration/test_config_decryption/configs/config_subnodes.xml new file mode 100644 index 000000000000..b0e519ff546f --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_subnodes.xml @@ -0,0 +1,10 @@ + + + + 00112233445566778899aabbccddeeff + + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + + diff --git a/tests/integration/test_config_decryption/configs/config_wrong_method.xml b/tests/integration/test_config_decryption/configs/config_wrong_method.xml new file mode 100644 index 000000000000..b452ce6374c3 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_wrong_method.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py new file mode 100644 index 000000000000..c01f5050b00a --- /dev/null +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -0,0 +1,34 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +def start_clickhouse(config, err_msg): + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance("node", main_configs=[config]) + caught_exception = "" + try: + cluster.start() + except Exception as e: + caught_exception = str(e) + assert caught_exception.find(err_msg) != -1 + + +def test_wrong_method(): + start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption Method") + + +def test_invalid_chars(): + start_clickhouse( + "configs/config_invalid_chars.xml", + "Cannot read encrypted text, check for valid characters", + ) + + +def test_no_encryption_codecs(): + start_clickhouse( + "configs/config_no_encryption_codecs.xml", "There is no key 0 in config" + ) + + +def test_subnodes(): + start_clickhouse("configs/config_subnodes.xml", "should have only one text node") From 906db0318dee9d08a8af603ab0400143578e4f3d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Jun 2023 19:45:43 +0000 Subject: [PATCH 033/226] Remove AST based optimization duplicate_order_by_and_distinct It was quite some time ago since it was replaced by plan level optimizations: - query_plan_remove_redundant_sorting - query_plan_remove_redundant_distinct --- src/Core/Settings.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 53 -------- ..._duplicate_order_by_and_distinct.reference | 58 --------- .../01305_duplicate_order_by_and_distinct.sql | 123 ------------------ ...t_optimize_for_distributed_table.reference | 4 - ...istinct_optimize_for_distributed_table.sql | 46 ------- 6 files changed, 1 insertion(+), 285 deletions(-) delete mode 100644 tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference delete mode 100644 tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql delete mode 100644 tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference delete mode 100644 tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3a23127e2fdd..c53bed2007ae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -526,7 +526,6 @@ class IColumn; M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_or_like_chain, false, "Optimize multiple OR LIKE into multiMatchAny. This optimization should not be enabled by default, because it defies index analysis in some cases.", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ - M(Bool, optimize_duplicate_order_by_and_distinct, false, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \ M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ @@ -818,6 +817,7 @@ class IColumn; MAKE_OBSOLETE(M, Seconds, drain_timeout, 3) \ MAKE_OBSOLETE(M, UInt64, backup_threads, 16) \ MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ + MAKE_OBSOLETE(M, Bool, optimize_duplicate_order_by_and_distinct, false) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c38b3c790265..b6b45c664f94 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -288,13 +288,6 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query) elems = std::move(unique_elems); } -/// Optimize duplicate ORDER BY -void optimizeDuplicateOrderBy(ASTPtr & query, ContextPtr context) -{ - DuplicateOrderByVisitor::Data order_by_data{context}; - DuplicateOrderByVisitor(order_by_data).visit(query); -} - /// Return simple subselect (without UNIONs or JOINs or SETTINGS) if any const ASTSelectQuery * getSimpleSubselect(const ASTSelectQuery & select) { @@ -378,41 +371,6 @@ std::unordered_set getDistinctNames(const ASTSelectQuery & select) return names; } -/// Remove DISTINCT from query if columns are known as DISTINCT from subquery -void optimizeDuplicateDistinct(ASTSelectQuery & select) -{ - if (!select.select() || select.select()->children.empty()) - return; - - const ASTSelectQuery * subselect = getSimpleSubselect(select); - if (!subselect) - return; - - std::unordered_set distinct_names = getDistinctNames(*subselect); - std::unordered_set selected_names; - - /// Check source column names from select list (ignore aliases and table names) - for (const auto & id : select.select()->children) - { - const auto * identifier = id->as(); - if (!identifier) - return; - - const String & name = identifier->shortName(); - if (!distinct_names.contains(name)) - return; /// Not a distinct column, keep DISTINCT for it. - - selected_names.emplace(name); - } - - /// select columns list != distinct columns list - /// SELECT DISTINCT a FROM (SELECT DISTINCT a, b FROM ...)) -- cannot remove DISTINCT - if (selected_names.size() != distinct_names.size()) - return; - - select.distinct = false; -} - /// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression, /// has a single argument and not an aggregate functions. void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, ContextPtr context, @@ -811,17 +769,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, && !select_query->group_by_with_cube) optimizeAggregateFunctionsOfGroupByKeys(select_query, query); - /// Remove duplicate ORDER BY and DISTINCT from subqueries. - if (settings.optimize_duplicate_order_by_and_distinct) - { - optimizeDuplicateOrderBy(query, context); - - /// DISTINCT has special meaning in Distributed query with enabled distributed_group_by_no_merge - /// TODO: disable Distributed/remote() tables only - if (!settings.distributed_group_by_no_merge) - optimizeDuplicateDistinct(*select_query); - } - /// Remove functions from ORDER BY if its argument is also in ORDER BY if (settings.optimize_redundant_functions_in_order_by) optimizeRedundantFunctionsInOrderBy(select_query, context); diff --git a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference deleted file mode 100644 index 10f8bbfd392f..000000000000 --- a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference +++ /dev/null @@ -1,58 +0,0 @@ -SELECT number -FROM -( - SELECT number - FROM - ( - SELECT DISTINCT number - FROM numbers(3) - ) -) -ORDER BY number ASC -0 -1 -2 -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM - ( - SELECT DISTINCT number - FROM numbers(3) - ORDER BY number ASC - ) - ORDER BY number ASC -) -ORDER BY number ASC -0 -1 -2 -SELECT number -FROM -( - SELECT number - FROM - ( - SELECT DISTINCT number % 2 AS number - FROM numbers(3) - ) -) -ORDER BY number ASC -0 -1 -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM - ( - SELECT DISTINCT number % 2 AS number - FROM numbers(3) - ORDER BY number ASC - ) - ORDER BY number ASC -) -ORDER BY number ASC -0 -1 diff --git a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql deleted file mode 100644 index 3b13b208eb5b..000000000000 --- a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql +++ /dev/null @@ -1,123 +0,0 @@ -set optimize_duplicate_order_by_and_distinct = 1; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 0; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 1; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 0; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; diff --git a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference b/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference deleted file mode 100644 index 44e0be8e3569..000000000000 --- a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql b/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql deleted file mode 100644 index 8ef1273c855f..000000000000 --- a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql +++ /dev/null @@ -1,46 +0,0 @@ --- Tags: distributed - -set query_plan_remove_redundant_distinct = 1; -set optimize_duplicate_order_by_and_distinct = 0; -SET distributed_group_by_no_merge = 0; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 - SETTINGS distributed_group_by_no_merge = 1 -); - -SET distributed_group_by_no_merge = 1; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 -); - -set optimize_duplicate_order_by_and_distinct = 0; -SET distributed_group_by_no_merge = 0; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 - SETTINGS distributed_group_by_no_merge = 1 -); - -SET distributed_group_by_no_merge = 1; -set optimize_duplicate_order_by_and_distinct = 0; -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 -); From a4e982442f4a3d6b3007b432f8e0b6211e9aa4e7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:13:59 +0000 Subject: [PATCH 034/226] Update documentation --- docs/en/operations/configuration-files.md | 36 +++++++++++++++++++++++ docs/ru/operations/configuration-files.md | 36 +++++++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index b3583e156ad1..b5d52acca49b 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -54,6 +54,42 @@ XML substitution example: Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. +## Decryption {#decryption} + +Elements with text nodes may be encrypted with [encryption codecs](../../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. + +Example: + +```xml + + + + 00112233445566778899aabbccddeeff + + + + admin + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + + +``` + +To get the encrypted value `encrypt_decrypt` example application may be used. + +Example: + +``` bash +./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV abcd +``` + +``` text +961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 +``` + +:::note +The decryption is executed after creation of preprocessed configuration file. It means that elements with `encryption_codec` attribute in the preprocessed configuration file are encrypted. But the values of corresponding parameters in server's memory are decrypted. +::: + ## User Settings {#user-settings} The `config.xml` file can specify a separate config with user settings, profiles, and quotas. The relative path to this config is set in the `users_config` element. By default, it is `users.xml`. If `users_config` is omitted, the user settings, profiles, and quotas are specified directly in `config.xml`. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 2b824ce91bd9..96512fbbe231 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -85,6 +85,42 @@ $ cat /etc/clickhouse-server/users.d/alice.xml Сервер следит за изменениями конфигурационных файлов, а также файлов и ZooKeeper-узлов, которые были использованы при выполнении подстановок и переопределений, и перезагружает настройки пользователей и кластеров на лету. То есть, можно изменять кластера, пользователей и их настройки без перезапуска сервера. +## Расшифровка {#decryption} + +Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../../sql-reference/statements/create/table.md#encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. + +Пример: + +```xml + + + + 00112233445566778899aabbccddeeff + + + + admin + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + + +``` + +Чтобы получить зашифрованное значение может быть использовано приложение-пример `encrypt_decrypt` . + +Пример: + +``` bash +./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV abcd +``` + +``` text +961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 +``` + +:::note +Расшифровка выполняется после создания конфигурационного файла предобработки. Это означает что элементы с аттрибутом `encryption_codec` в конфигурационном файле предобработки зашифрованы. Но значения соответствующих параметров в памяти сервера расшифрованы. +::: + ## Примеры записи конфигурации на YAML {#example} Здесь можно рассмотреть пример реальной конфигурации записанной на YAML: [config.yaml.example](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.yaml.example). From b5d4ad583f3741f87843f51c56ccc41b91833523 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:35:55 +0000 Subject: [PATCH 035/226] Small code style improvements --- src/Common/Config/ConfigProcessor.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index df25a9a3825a..9548bf33b7bb 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -26,9 +26,9 @@ #include #include #include +#include #include #include -#include #include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -194,7 +194,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const DB::Memory<> memory; memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); - std::string encrypted_value = std::string(memory.data(), bytes_written); + auto encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); return hex_value; @@ -224,7 +224,7 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { - for (Node * node = config_root->firstChild(); node;) + for (Node * node = config_root->firstChild(); node; node = node->nextSibling()) { if (node->nodeType() == Node::ELEMENT_NODE) { @@ -244,7 +244,6 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) } decryptRecursive(node); } - node = node->nextSibling(); } } From f55623aa2d23fda63f2b19720f4035568a4595a4 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:46:43 +0000 Subject: [PATCH 036/226] Use anonymous namespace for getEncryptionMethod() --- src/Common/Config/ConfigProcessor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 9548bf33b7bb..17abc3d161d7 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -47,6 +47,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + /// Get method for string name. Throw exception for wrong name EncryptionMethod getEncryptionMethod(const std::string & name) { @@ -58,6 +61,8 @@ EncryptionMethod getEncryptionMethod(const std::string & name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); } +} + /// For cutting preprocessed path to this base static std::string main_config_path; From 14dfebba49543378b80716cffb5aaea7dcc7fbf7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 13:35:11 +0000 Subject: [PATCH 037/226] Fix links in MD --- docs/en/operations/configuration-files.md | 2 +- docs/ru/operations/configuration-files.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index b5d52acca49b..71d5885058ab 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -56,7 +56,7 @@ Substitutions can also be performed from ZooKeeper. To do this, specify the attr ## Decryption {#decryption} -Elements with text nodes may be encrypted with [encryption codecs](../../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. +Elements with text nodes may be encrypted with [encryption codecs](../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. Example: diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 96512fbbe231..df50d9009197 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -87,7 +87,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## Расшифровка {#decryption} -Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../../sql-reference/statements/create/table.md#encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. +Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../sql-reference/statements/create/table.md#create-query-encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. Пример: From 3d64cf4423b9fb4b935786eca392875d3b66c17c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 15:40:32 +0000 Subject: [PATCH 038/226] Add dbms in cmake --- src/Common/Config/CMakeLists.txt | 2 ++ utils/config-processor/CMakeLists.txt | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index ec7bdd101964..fdcba5d4a4b7 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -15,6 +15,7 @@ target_link_libraries(clickhouse_common_config Poco::XML PRIVATE string_utils + dbms ) add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) @@ -25,6 +26,7 @@ target_link_libraries(clickhouse_common_config_no_zookeeper_log Poco::XML PRIVATE string_utils + dbms ) if (TARGET ch_contrib::yaml_cpp) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 80c3535ef4eb..53b6163ba871 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE dbms) +target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) From f830f246627327f5ede014d08da7af8964f4cb7c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 15 Jun 2023 14:31:21 +0000 Subject: [PATCH 039/226] Revert "Add dbms in cmake" This reverts commit 3d64cf4423b9fb4b935786eca392875d3b66c17c. --- src/Common/Config/CMakeLists.txt | 2 -- utils/config-processor/CMakeLists.txt | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index fdcba5d4a4b7..ec7bdd101964 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -15,7 +15,6 @@ target_link_libraries(clickhouse_common_config Poco::XML PRIVATE string_utils - dbms ) add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) @@ -26,7 +25,6 @@ target_link_libraries(clickhouse_common_config_no_zookeeper_log Poco::XML PRIVATE string_utils - dbms ) if (TARGET ch_contrib::yaml_cpp) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 53b6163ba871..80c3535ef4eb 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) +target_link_libraries(config-processor PRIVATE dbms) From 98597a3b422c4b145ed44ecfeb305840643dfb1b Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 15 Jun 2023 14:50:52 +0000 Subject: [PATCH 040/226] Add USE_SSL --- src/Common/Config/ConfigProcessor.cpp | 16 ++++++++++++++++ src/Common/Config/ConfigProcessor.h | 4 ++++ src/Common/Config/ConfigReloader.cpp | 3 +++ src/Common/examples/CMakeLists.txt | 2 ++ src/Daemon/BaseDaemon.cpp | 3 +++ 5 files changed, 28 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 17abc3d161d7..11b45977322c 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -26,10 +26,13 @@ #include #include #include + +#if USE_SSL #include #include #include #include +#endif #define PREPROCESSED_SUFFIX "-preprocessed" @@ -44,9 +47,12 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int CANNOT_LOAD_CONFIG; +#if USE_SSL extern const int BAD_ARGUMENTS; +#endif } +#if USE_SSL namespace { @@ -63,6 +69,8 @@ EncryptionMethod getEncryptionMethod(const std::string & name) } +#endif + /// For cutting preprocessed path to this base static std::string main_config_path; @@ -192,6 +200,8 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } +#if USE_SSL + std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); @@ -252,6 +262,8 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) } } +#endif + void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, const Node * with_root) { const NodeListPtr with_nodes = with_root->childNodes(); @@ -781,6 +793,8 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } +#if USE_SSL + void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); @@ -789,6 +803,8 @@ void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); } +#endif + void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) { try diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index c9b227863f06..8d7caa9e9c8c 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,6 +92,7 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); +#if USE_SSL /// Encrypt text value std::string encryptValue(const std::string & codec_name, const std::string & value); @@ -100,6 +101,7 @@ class ConfigProcessor /// Decrypt nodes in config with specified encryption attributes void decryptConfig(LoadedConfig & loaded_config); +#endif /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ @@ -133,7 +135,9 @@ class ConfigProcessor using NodePtr = Poco::AutoPtr; +#if USE_SSL void decryptRecursive(Poco::XML::Node * config_root); +#endif void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index a4d2cb3d3053..45192d2d281f 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -130,7 +130,10 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); + +#if USE_SSL config_processor.decryptConfig(loaded_config); +#endif /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 12a2b59ff775..0965a07761b5 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -83,5 +83,7 @@ endif() clickhouse_add_executable (interval_tree interval_tree.cpp) target_link_libraries (interval_tree PRIVATE dbms) +if (ENABLE_SSL) clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) target_link_libraries (encrypt_decrypt PRIVATE dbms) +endif() diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 4b1cd4e036ea..5747cfe158d9 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -663,7 +663,10 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); + +#if USE_SSL DB::ConfigProcessor(config_path).decryptConfig(loaded_config); +#endif /// Write core dump on crash. { From 55013342d18ba3363695358d02e693b82872a93b Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 16 Jun 2023 13:26:45 +0000 Subject: [PATCH 041/226] Fix code align in cmake --- src/Common/examples/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 0965a07761b5..90a238c9800c 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -84,6 +84,6 @@ clickhouse_add_executable (interval_tree interval_tree.cpp) target_link_libraries (interval_tree PRIVATE dbms) if (ENABLE_SSL) -clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) -target_link_libraries (encrypt_decrypt PRIVATE dbms) + clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) + target_link_libraries (encrypt_decrypt PRIVATE dbms) endif() From f026cf17a3c8e8a69d3837237e9c9dac08937644 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 16 Jun 2023 15:26:58 +0000 Subject: [PATCH 042/226] Fix building with BUILD_STANDALONE_KEEPER --- programs/keeper/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index e5d56023f7b7..aa90ba78f443 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -84,6 +84,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedReadBufferFromFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressedWriteBuffer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecEncrypted.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecLZ4.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecMultiple.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CompressionCodecNone.cpp From 5bba0ff6984de2baf8e94efef11eafa036090ad5 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 16 Jun 2023 20:13:36 +0000 Subject: [PATCH 043/226] Fix build of keeper-bench --- utils/config-processor/CMakeLists.txt | 6 +++++- utils/keeper-bench/CMakeLists.txt | 7 ++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 80c3535ef4eb..4394083a1c36 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,6 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE dbms) +if (ENABLE_SSL) + target_link_libraries(config-processor PRIVATE dbms) +else () + target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) +endif () diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index 49ce20682467..e8daec9e1642 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -4,4 +4,9 @@ if (NOT TARGET ch_contrib::rapidjson) endif () clickhouse_add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -target_link_libraries(keeper-bench PRIVATE clickhouse_common_config_no_zookeeper_log ch_contrib::rapidjson) +if (ENABLE_SSL) + target_link_libraries(keeper-bench PRIVATE dbms) +else () + target_link_libraries(keeper-bench PRIVATE clickhouse_common_config_no_zookeeper_log) +endif () +target_link_libraries(keeper-bench PRIVATE ch_contrib::rapidjson) From af2be06c42395d3f29d7325e28e89e7dc1aad12d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 16 Jun 2023 20:56:26 +0000 Subject: [PATCH 044/226] Remove test --- ..._duplicate_distinct_optimization.reference | 136 ------------------ .../01455_duplicate_distinct_optimization.sql | 32 ----- 2 files changed, 168 deletions(-) delete mode 100644 tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference delete mode 100644 tests/queries/0_stateless/01455_duplicate_distinct_optimization.sql diff --git a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference b/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference deleted file mode 100644 index 82e887e1b929..000000000000 --- a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.reference +++ /dev/null @@ -1,136 +0,0 @@ -SELECT DISTINCT number -FROM numbers(1) -SELECT number -FROM -( - SELECT DISTINCT number - FROM numbers(1) -) -SELECT DISTINCT number * 2 -FROM -( - SELECT DISTINCT - number * 2, - number - FROM numbers(1) -) -SELECT number -FROM -( - SELECT DISTINCT number * 2 AS number - FROM numbers(1) -) -SELECT - b, - a -FROM -( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) -) -SELECT DISTINCT a -FROM -( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) -) -SELECT a -FROM -( - SELECT DISTINCT a - FROM - ( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) - ) -) -SELECT DISTINCT a -FROM -( - SELECT - a, - b - FROM - ( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) - ) -) -SELECT - a, - b -FROM -( - SELECT - b, - a - FROM - ( - SELECT DISTINCT - number AS a, - number AS b - FROM numbers(1) - ) -) -SELECT - a, - b -FROM -( - SELECT - b, - a, - a + b - FROM - ( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) - ) -) -SELECT DISTINCT a -FROM -( - SELECT a - FROM - ( - SELECT DISTINCT - number % 2 AS a, - number % 3 AS b - FROM numbers(100) - ) -) -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM numbers(1) -) AS t1 -CROSS JOIN numbers(2) AS t2 -SELECT number -FROM -( - SELECT DISTINCT number - FROM numbers(1) AS t1 - CROSS JOIN numbers(2) AS t2 -) -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM numbers(1) - UNION ALL - SELECT DISTINCT number - FROM numbers(2) -) -0 -1 diff --git a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.sql b/tests/queries/0_stateless/01455_duplicate_distinct_optimization.sql deleted file mode 100644 index 6fbf80a4dc3e..000000000000 --- a/tests/queries/0_stateless/01455_duplicate_distinct_optimization.sql +++ /dev/null @@ -1,32 +0,0 @@ -SET optimize_duplicate_order_by_and_distinct = 1; - -EXPLAIN SYNTAX SELECT DISTINCT number FROM numbers(1); -EXPLAIN SYNTAX SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1)); -EXPLAIN SYNTAX SELECT DISTINCT number * 2 FROM (SELECT DISTINCT number * 2, number FROM numbers(1)); -EXPLAIN SYNTAX SELECT DISTINCT number FROM (SELECT DISTINCT number * 2 AS number FROM numbers(1)); -EXPLAIN SYNTAX SELECT DISTINCT b, a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100)); -EXPLAIN SYNTAX SELECT DISTINCT a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100)); -EXPLAIN SYNTAX SELECT DISTINCT a FROM (SELECT DISTINCT a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100))); -EXPLAIN SYNTAX SELECT DISTINCT a FROM (SELECT DISTINCT a, b FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100))); -EXPLAIN SYNTAX SELECT DISTINCT a, b FROM (SELECT DISTINCT b, a FROM (SELECT DISTINCT number a, number b FROM numbers(1))); -EXPLAIN SYNTAX SELECT DISTINCT a, b FROM (SELECT b, a, a + b FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100))); -EXPLAIN SYNTAX SELECT DISTINCT a FROM (SELECT a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100))); -EXPLAIN SYNTAX SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1)) t1 CROSS JOIN numbers(2) t2; -EXPLAIN SYNTAX SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1) t1 CROSS JOIN numbers(2) t2); - -EXPLAIN SYNTAX SELECT DISTINCT number FROM -( - (SELECT DISTINCT number FROM numbers(1)) - UNION ALL - (SELECT DISTINCT number FROM numbers(2)) -); - --- - -SELECT DISTINCT number FROM -( - (SELECT DISTINCT number FROM numbers(1)) - UNION ALL - (SELECT DISTINCT number FROM numbers(2)) -) -ORDER BY number; From 85d86fec8337e347b519ebd7318012e83af109ec Mon Sep 17 00:00:00 2001 From: sanjam Date: Thu, 22 Jun 2023 13:38:50 +0000 Subject: [PATCH 045/226] external_aggregation_fix --- .../HashTable/TwoLevelStringHashTable.h | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index ea1914348b27..ee6dcd05d9ae 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -113,14 +113,20 @@ class TwoLevelStringHashTable : private boost::noncopyable if ((reinterpret_cast(p) & 2048) == 0) { memcpy(&n[0], p, 8); - n[0] &= -1ULL >> s; - } + if constexpr (std::endian::native == std::endian::little) + n[0] &= -1ULL >> s; + else + n[0] &= -1ULL << s; + } else { const char * lp = x.data + x.size - 8; memcpy(&n[0], lp, 8); - n[0] >>= s; - } + if constexpr (std::endian::native == std::endian::little) + n[0] >>= s; + else + n[0] <<= s; + } auto res = hash(k8); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); @@ -131,8 +137,11 @@ class TwoLevelStringHashTable : private boost::noncopyable memcpy(&n[0], p, 8); const char * lp = x.data + x.size - 8; memcpy(&n[1], lp, 8); - n[1] >>= s; - auto res = hash(k16); + if constexpr (std::endian::native == std::endian::little) + n[1] >>= s; + else + n[1] <<= s; + auto res = hash(k16); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m2, k16, res); @@ -142,8 +151,11 @@ class TwoLevelStringHashTable : private boost::noncopyable memcpy(&n[0], p, 16); const char * lp = x.data + x.size - 8; memcpy(&n[2], lp, 8); - n[2] >>= s; - auto res = hash(k24); + if constexpr (std::endian::native == std::endian::little) + n[2] >>= s; + else + n[2] <<= s; + auto res = hash(k24); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m3, k24, res); From cf082f2f9a68c21241c9b6667a8e4241da220601 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Jun 2023 17:24:43 +0000 Subject: [PATCH 046/226] Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 8 +- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 5 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 10 +- .../AzureBlobStorage/AzureObjectStorage.h | 5 + src/IO/Progress.h | 9 +- src/IO/ReadBufferFromFileBase.cpp | 2 +- src/IO/ReadBufferFromFileBase.h | 2 +- src/IO/ReadBufferFromS3.cpp | 15 +- src/IO/ReadBufferFromS3.h | 6 +- src/IO/ReadWriteBufferFromHTTP.cpp | 15 +- src/IO/ReadWriteBufferFromHTTP.h | 5 +- .../Executors/ExecutionThreadContext.cpp | 3 + src/Processors/IProcessor.h | 1 + src/Processors/ISource.h | 1 + src/Processors/Sources/RemoteSource.cpp | 2 + src/QueryPipeline/ReadProgressCallback.cpp | 12 ++ src/QueryPipeline/ReadProgressCallback.h | 3 + src/Storages/HDFS/ReadBufferFromHDFS.cpp | 29 +++- src/Storages/HDFS/ReadBufferFromHDFS.h | 6 +- src/Storages/HDFS/StorageHDFS.cpp | 74 ++++----- src/Storages/HDFS/StorageHDFS.h | 6 - src/Storages/StorageAzureBlob.cpp | 83 ++++------ src/Storages/StorageAzureBlob.h | 18 +-- src/Storages/StorageFile.cpp | 32 +--- src/Storages/StorageS3.cpp | 150 ++++++++---------- src/Storages/StorageS3.h | 29 +--- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageURL.cpp | 37 ++--- src/Storages/StorageURL.h | 8 +- 29 files changed, 267 insertions(+), 311 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 129bb97be097..6a328de03418 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace ProfileEvents @@ -36,7 +37,8 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( size_t max_single_download_retries_, bool use_external_buffer_, bool restricted_seek_, - size_t read_until_position_) + size_t read_until_position_, + std::function progress_callback_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size, nullptr, 0) , blob_container_client(blob_container_client_) , path(path_) @@ -47,6 +49,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( , use_external_buffer(use_external_buffer_) , restricted_seek(restricted_seek_) , read_until_position(read_until_position_) + , progress_callback(progress_callback_) { if (!use_external_buffer) { @@ -127,6 +130,9 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() if (bytes_read == 0) return false; + if (progress_callback) + progress_callback(FileProgress(bytes_read)); + BufferBase::set(data_ptr, bytes_read, 0); offset += bytes_read; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 4e21f5436536..6f683dcf1cec 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -25,7 +25,8 @@ class ReadBufferFromAzureBlobStorage : public ReadBufferFromFileBase size_t max_single_download_retries_, bool use_external_buffer_ = false, bool restricted_seek_ = false, - size_t read_until_position_ = 0); + size_t read_until_position_ = 0, + std::function progress_callback_ = {}); off_t seek(off_t off, int whence) override; @@ -74,6 +75,8 @@ class ReadBufferFromAzureBlobStorage : public ReadBufferFromFileBase size_t data_capacity; Poco::Logger * log = &Poco::Logger::get("ReadBufferFromAzureBlobStorage"); + + std::function progress_callback; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index dbb418510532..982c376404aa 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -191,7 +191,7 @@ std::unique_ptr AzureObjectStorage::readObject( /// NOLI return std::make_unique( client.get(), object.remote_path, patchSettings(read_settings), settings_ptr->max_single_read_retries, - settings_ptr->max_single_download_retries); + settings_ptr->max_single_download_retries, false, false, 0, progress_callback); } std::unique_ptr AzureObjectStorage::readObjects( /// NOLINT @@ -216,7 +216,8 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL settings_ptr->max_single_download_retries, /* use_external_buffer */true, /* restricted_seek */true, - read_until_position); + read_until_position, + progress_callback); }; switch (read_settings.remote_fs_method) @@ -390,6 +391,11 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std ); } +void AzureObjectStorage::setProgressCallback(const ContextPtr & context) +{ + progress_callback = context->getFileProgressCallback(); +} + } #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index b5f81cef2350..ee144cdd56ec 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -123,6 +123,9 @@ class AzureObjectStorage : public IObjectStorage bool isRemote() const override { return true; } + /// Set progress callback to read buffer while reading from storage. + void setProgressCallback(const ContextPtr & context); + private: const String name; /// client used to access the files in the Blob Storage cloud @@ -132,6 +135,8 @@ class AzureObjectStorage : public IObjectStorage Poco::Logger * log; DataSourceDescription data_source_description; + + std::function progress_callback; }; } diff --git a/src/IO/Progress.h b/src/IO/Progress.h index c21b1b854b0b..a68ff9bc5c22 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -40,9 +40,10 @@ struct ReadProgress UInt64 read_rows = 0; UInt64 read_bytes = 0; UInt64 total_rows_to_read = 0; + UInt64 total_bytes_to_read = 0; - ReadProgress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0) - : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_) {} + ReadProgress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) + : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_), total_bytes_to_read(total_bytes_to_read_) {} }; struct WriteProgress @@ -98,8 +99,8 @@ struct Progress Progress() = default; - Progress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0) - : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_) {} + Progress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) + : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_), total_bytes_to_read(total_bytes_to_read_) {} explicit Progress(ReadProgress read_progress) : read_rows(read_progress.read_rows), read_bytes(read_progress.read_bytes), total_rows_to_read(read_progress.total_rows_to_read) {} diff --git a/src/IO/ReadBufferFromFileBase.cpp b/src/IO/ReadBufferFromFileBase.cpp index 4181615bc52f..4ac3f984f788 100644 --- a/src/IO/ReadBufferFromFileBase.cpp +++ b/src/IO/ReadBufferFromFileBase.cpp @@ -42,7 +42,7 @@ void ReadBufferFromFileBase::setProgressCallback(ContextPtr context) setProfileCallback([file_progress_callback](const ProfileInfo & progress) { - file_progress_callback(FileProgress(progress.bytes_read, 0)); + file_progress_callback(FileProgress(progress.bytes_read)); }); } diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index b77db29bc238..2abdf883ab09 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -52,7 +52,7 @@ class ReadBufferFromFileBase : public BufferWithOwnMemory, p size_t getFileSize() override; - void setProgressCallback(ContextPtr context); + virtual void setProgressCallback(ContextPtr context); protected: std::optional file_size; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index d1cb1ec9ab03..36ff81a85d42 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -162,12 +163,13 @@ bool ReadBufferFromS3::nextImpl() offset += working_buffer.size(); if (read_settings.remote_throttler) read_settings.remote_throttler->add(working_buffer.size(), ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - + if (progress_callback) + progress_callback(FileProgress(working_buffer.size())); return true; } -size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & custom_progress_callback) { if (n == 0) return 0; @@ -184,7 +186,9 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons auto result = sendRequest(range_begin, range_begin + n - 1); std::istream & istr = result.GetBody(); - size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, progress_callback); + size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, custom_progress_callback); + if (progress_callback) + progress_callback(FileProgress(bytes, 0)); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes); @@ -415,6 +419,11 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin } } +void ReadBufferFromS3::setProgressCallback(DB::ContextPtr context) +{ + progress_callback = context->getFileProgressCallback(); +} + } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 0f665861a1ea..824038c7af01 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -77,10 +77,12 @@ class ReadBufferFromS3 : public ReadBufferFromFileBase String getFileName() const override { return bucket + "/" + key; } - size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & custom_progress_callback) override; bool supportsReadAt() override { return true; } + void setProgressCallback(ContextPtr context) override; + private: std::unique_ptr initialize(); @@ -100,6 +102,8 @@ class ReadBufferFromS3 : public ReadBufferFromFileBase /// There is different seek policy for disk seek and for non-disk seek /// (non-disk seek is applied for seekable input formats: orc, arrow, parquet). bool restricted_seek; + + std::function progress_callback; }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index cf1159bfb4b1..7bd7f4a9b8ed 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -587,11 +587,13 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() internal_buffer = impl->buffer(); working_buffer = internal_buffer; offset_from_begin_pos += working_buffer.size(); + if (progress_callback) + progress_callback(FileProgress(working_buffer.size())); return true; } template -size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) +size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & custom_progress_callback) { /// Caller must have checked supportsReadAt(). /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. @@ -633,8 +635,9 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si toString(response.getStatus()), uri_.toString(), offset, offset + n); bool cancelled; - size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); - + size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, custom_progress_callback, &cancelled); + if (progress_callback) + progress_callback(FileProgress(r)); return r; } catch (const Poco::Exception & e) @@ -780,6 +783,12 @@ void ReadWriteBufferFromHTTPBase::setNextCallback(NextCallb next_callback(count()); } +template +void ReadWriteBufferFromHTTPBase::setProgressCallback(std::function file_progress_callback_) +{ + progress_callback = file_progress_callback_; +} + template const std::string & ReadWriteBufferFromHTTPBase::getCompressionMethod() const { return content_encoding; } diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 2d2ae5fe724e..18bd31fcdce2 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -92,6 +92,7 @@ namespace detail HTTPHeaderEntries http_header_entries; const RemoteHostFilter * remote_host_filter = nullptr; std::function next_callback; + std::function progress_callback; size_t buffer_size; bool use_external_buffer; @@ -176,7 +177,7 @@ namespace detail bool nextImpl() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & custom_progress_callback) override; off_t getPosition() override; @@ -199,6 +200,8 @@ namespace detail /// passed through the buffer void setNextCallback(NextCallback next_callback_); + void setProgressCallback(std::function progress_callback_); + const std::string & getCompressionMethod() const; std::optional getLastModificationTime(); diff --git a/src/Processors/Executors/ExecutionThreadContext.cpp b/src/Processors/Executors/ExecutionThreadContext.cpp index 794f478b272b..0fa7e0b552f9 100644 --- a/src/Processors/Executors/ExecutionThreadContext.cpp +++ b/src/Processors/Executors/ExecutionThreadContext.cpp @@ -56,6 +56,9 @@ static void executeJob(ExecutingGraph::Node * node, ReadProgressCallback * read_ if (read_progress->counters.total_rows_approx) read_progress_callback->addTotalRowsApprox(read_progress->counters.total_rows_approx); + if (read_progress->counters.total_bytes) + read_progress_callback->addTotalBytes(read_progress->counters.total_bytes); + if (!read_progress_callback->onProgress(read_progress->counters.read_rows, read_progress->counters.read_bytes, read_progress->limits)) node->processor->cancel(); } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 34322acb2af8..c6bef1868775 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -343,6 +343,7 @@ class IProcessor uint64_t read_rows = 0; uint64_t read_bytes = 0; uint64_t total_rows_approx = 0; + uint64_t total_bytes = 0; }; struct ReadProgress diff --git a/src/Processors/ISource.h b/src/Processors/ISource.h index 292f79ba3486..2593a241c638 100644 --- a/src/Processors/ISource.h +++ b/src/Processors/ISource.h @@ -43,6 +43,7 @@ class ISource : public IProcessor std::optional getReadProgress() final; void addTotalRowsApprox(size_t value) { read_progress.total_rows_approx += value; } + void addTotalBytes(size_t value) { read_progress.total_bytes += value; } }; using SourcePtr = std::shared_ptr; diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 310a1d33e282..74ab36490686 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -77,6 +77,8 @@ std::optional RemoteSource::tryGenerate() { if (value.total_rows_to_read) addTotalRowsApprox(value.total_rows_to_read); + if (value.total_bytes_to_read) + addTotalBytes(value.total_bytes_to_read); progress(value.read_rows, value.read_bytes); }); diff --git a/src/QueryPipeline/ReadProgressCallback.cpp b/src/QueryPipeline/ReadProgressCallback.cpp index 0f50d56f1a58..4d7c7aa0f2a7 100644 --- a/src/QueryPipeline/ReadProgressCallback.cpp +++ b/src/QueryPipeline/ReadProgressCallback.cpp @@ -63,6 +63,18 @@ bool ReadProgressCallback::onProgress(uint64_t read_rows, uint64_t read_bytes, c process_list_elem->updateProgressIn(total_rows_progress); } + size_t bytes = 0; + if ((bytes = total_bytes.exchange(0)) != 0) + { + Progress total_bytes_progress = {0, 0, 0, bytes}; + + if (progress_callback) + progress_callback(total_bytes_progress); + + if (process_list_elem) + process_list_elem->updateProgressIn(total_bytes_progress); + } + Progress value {read_rows, read_bytes}; if (progress_callback) diff --git a/src/QueryPipeline/ReadProgressCallback.h b/src/QueryPipeline/ReadProgressCallback.h index 08f2f9fc99b0..5dbf3344bdf5 100644 --- a/src/QueryPipeline/ReadProgressCallback.h +++ b/src/QueryPipeline/ReadProgressCallback.h @@ -23,6 +23,7 @@ class ReadProgressCallback void setProcessListElement(QueryStatusPtr elem); void setProgressCallback(const ProgressCallback & callback) { progress_callback = callback; } void addTotalRowsApprox(size_t value) { total_rows_approx += value; } + void addTotalBytes(size_t value) { total_bytes += value; } /// Skip updating profile events. /// For merges in mutations it may need special logic, it's done inside ProgressCallback. @@ -37,6 +38,8 @@ class ReadProgressCallback /// The approximate total number of rows to read. For progress bar. std::atomic_size_t total_rows_approx = 0; + /// The total number of bytes to read. For progress bar. + std::atomic_size_t total_bytes = 0; std::mutex limits_and_quotas_mutex; Stopwatch total_stopwatch{CLOCK_MONOTONIC_COARSE}; /// Including waiting time diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index ee8e0764db08..2c2c5047cb15 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -3,6 +3,7 @@ #if USE_HDFS #include #include +#include #include #include #include @@ -42,19 +43,23 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory file_size; + explicit ReadBufferFromHDFSImpl( const std::string & hdfs_uri_, const std::string & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, size_t read_until_position_, - bool use_external_buffer_) + bool use_external_buffer_, + std::optional file_size_) : BufferWithOwnMemory(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri_, config_)) , read_settings(read_settings_) , read_until_position(read_until_position_) + , file_size(file_size_) { fs = createHDFSFS(builder.get()); fin = hdfsOpenFile(fs.get(), hdfs_file_path.c_str(), O_RDONLY, 0, 0, 0); @@ -70,12 +75,16 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemorymSize; + file_size = static_cast(file_info->mSize); + return *file_size; } bool nextImpl() override @@ -156,10 +165,11 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, size_t read_until_position_, - bool use_external_buffer_) + bool use_external_buffer_, + std::optional file_size_) : ReadBufferFromFileBase(read_settings_.remote_fs_buffer_size, nullptr, 0) , impl(std::make_unique( - hdfs_uri_, hdfs_file_path_, config_, read_settings_, read_until_position_, use_external_buffer_)) + hdfs_uri_, hdfs_file_path_, config_, read_settings_, read_until_position_, use_external_buffer_, file_size_)) , use_external_buffer(use_external_buffer_) { } @@ -188,7 +198,11 @@ bool ReadBufferFromHDFS::nextImpl() auto result = impl->next(); if (result) + { BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); /// use the buffer returned by `impl` + if (progress_callback) + progress_callback(FileProgress(working_buffer.size())); + } return result; } @@ -248,6 +262,11 @@ String ReadBufferFromHDFS::getFileName() const return impl->hdfs_file_path; } +void ReadBufferFromHDFS::setProgressCallback(DB::ContextPtr context) +{ + progress_callback = context->getFileProgressCallback(); +} + } #endif diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 6aed3ddff26e..3dce6a93cbac 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -29,7 +29,8 @@ struct ReadBufferFromHDFSImpl; const Poco::Util::AbstractConfiguration & config_, const ReadSettings & read_settings_, size_t read_until_position_ = 0, - bool use_external_buffer = false); + bool use_external_buffer = false, + std::optional file_size = std::nullopt); ~ReadBufferFromHDFS() override; @@ -47,9 +48,12 @@ struct ReadBufferFromHDFSImpl; String getFileName() const override; + void setProgressCallback(ContextPtr context) override; + private: std::unique_ptr impl; bool use_external_buffer; + std::function progress_callback; }; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 583c45a06334..79cda3050d6e 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -259,8 +259,13 @@ class HDFSSource::DisclosedGlobIterator::Impl { const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); uris = getPathsList(path_from_uri, uri_without_path, context_); + auto file_progress_callback = context_->getFileProgressCallback(); for (auto & elem : uris) + { elem.path = uri_without_path + elem.path; + if (file_progress_callback && elem.info) + file_progress_callback(FileProgress(0, elem.info->size)); + } uris_iter = uris.begin(); } @@ -281,37 +286,40 @@ class HDFSSource::DisclosedGlobIterator::Impl std::vector::iterator uris_iter; }; -class HDFSSource::URISIterator::Impl +class HDFSSource::URISIterator::Impl : WithContext { public: - explicit Impl(const std::vector & uris_, ContextPtr context) + explicit Impl(const std::vector & uris_, ContextPtr context_) + : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) { - auto path_and_uri = getPathFromUriAndUriWithoutPath(uris_[0]); - HDFSBuilderWrapper builder = createHDFSBuilder(path_and_uri.second + "/", context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - for (const auto & uri : uris_) - { - path_and_uri = getPathFromUriAndUriWithoutPath(uri); - if (!hdfsExists(fs.get(), path_and_uri.first.c_str())) - uris.push_back(uri); - } - uris_iter = uris.begin(); } StorageHDFS::PathWithInfo next() { - std::lock_guard lock(mutex); - if (uris_iter == uris.end()) + size_t current_index = index.fetch_add(1); + if (current_index >= uris.size()) return {"", {}}; - auto key = *uris_iter; - ++uris_iter; - return {key, {}}; + + auto uri = uris[current_index]; + auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); + HDFSBuilderWrapper builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); + auto fs = createHDFSFS(builder.get()); + auto * hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); + std::optional info; + if (hdfs_info) + { + info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; + if (file_progress_callback && hdfs_info) + file_progress_callback(FileProgress(0, hdfs_info->mSize)); + } + + return {uri, info}; } private: - std::mutex mutex; + std::atomic_size_t index = 0; Strings uris; - Strings::iterator uris_iter; + std::function file_progress_callback; }; HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) @@ -348,7 +356,7 @@ HDFSSource::HDFSSource( UInt64 max_block_size_, std::shared_ptr file_iterator_, ColumnsDescription columns_description_) - : ISource(getHeader(block_for_format_, requested_virtual_columns_)) + : ISource(getHeader(block_for_format_, requested_virtual_columns_), false) , WithContext(context_) , storage(std::move(storage_)) , block_for_format(block_for_format_) @@ -374,13 +382,17 @@ bool HDFSSource::initialize() continue; current_path = path_with_info.path; + std::optional file_size; + if (path_with_info.info) + file_size = path_with_info.info->size; const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); auto impl = std::make_unique( - uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); + uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings(), 0, false, file_size); if (!skip_empty_files || !impl->eof()) { + impl->setProgressCallback(getContext()); const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); break; @@ -389,15 +401,7 @@ bool HDFSSource::initialize() current_path = path_with_info.path; - if (path_with_info.info && path_with_info.info->size) - { - /// Adjust total_rows_approx_accumulated with new total size. - if (total_files_size) - total_rows_approx_accumulated = static_cast(std::ceil(static_cast(total_files_size + path_with_info.info->size) / total_files_size * total_rows_approx_accumulated)); - total_files_size += path_with_info.info->size; - } - - input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); + auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -434,14 +438,7 @@ Chunk HDFSSource::generate() { Columns columns = chunk.getColumns(); UInt64 num_rows = chunk.getNumRows(); - - if (num_rows && total_files_size) - { - size_t chunk_size = input_format->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } + progress(num_rows, 0); for (const auto & virtual_column : requested_virtual_columns) { @@ -465,7 +462,6 @@ Chunk HDFSSource::generate() reader.reset(); pipeline.reset(); - input_format.reset(); read_buf.reset(); if (!initialize()) diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 74801b68f73b..5a3b97a0e3c8 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -164,16 +164,10 @@ class HDFSSource : public ISource, WithContext ColumnsDescription columns_description; std::unique_ptr read_buf; - std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; String current_path; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; - size_t total_files_size = 0; - /// Recreate ReadBuffer and PullingPipelineExecutor for each file. bool initialize(); }; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 336c4eaed9b0..8e06ceda8858 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -617,13 +617,13 @@ Pipe StorageAzureBlob::read( /// Iterate through disclosed globs and make a source for each file iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, std::nullopt, - configuration.blob_path, query_info.query, virtual_block, local_context, nullptr); + configuration.blob_path, query_info.query, virtual_block, local_context, nullptr, local_context->getFileProgressCallback()); } else { iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, query_info.query, virtual_block, local_context, nullptr); + std::nullopt, query_info.query, virtual_block, local_context, nullptr, local_context->getFileProgressCallback()); } ColumnsDescription columns_description; @@ -794,15 +794,16 @@ StorageAzureBlobSource::Iterator::Iterator( ASTPtr query_, const Block & virtual_header_, ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_) + RelativePathsWithMetadata * outer_blobs_, + std::function file_progress_callback_) : WithContext(context_) , object_storage(object_storage_) , container(container_) - , keys(keys_) , blob_path_with_globs(blob_path_with_globs_) , query(query_) , virtual_header(virtual_header_) , outer_blobs(outer_blobs_) + , file_progress_callback(file_progress_callback_) { if (keys.has_value() && blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simultaneously it's a bug"); @@ -810,11 +811,10 @@ StorageAzureBlobSource::Iterator::Iterator( if (!keys.has_value() && !blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Both keys and glob mask are not specified"); - if (keys) + if (keys_) { Strings all_keys = *keys; - blobs_with_metadata.emplace(); /// Create a virtual block with one row to construct filter if (query && virtual_header && !all_keys.empty()) { @@ -843,29 +843,12 @@ StorageAzureBlobSource::Iterator::Iterator( all_keys = std::move(filtered_keys); } } - - for (auto && key : all_keys) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); - total_size += object_metadata.size_bytes; - blobs_with_metadata->emplace_back(RelativePathWithMetadata{key, object_metadata}); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata->back()); - } + keys = std::move(all_keys); } else { const String key_prefix = blob_path_with_globs->substr(0, blob_path_with_globs->find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == blob_path_with_globs->size()) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(*blob_path_with_globs); - blobs_with_metadata->emplace_back(*blob_path_with_globs, object_metadata); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata->back()); - return; - } + assert(key_prefix.size() != blob_path_with_globs->size()); object_storage_iterator = object_storage->iterate(key_prefix); @@ -888,13 +871,17 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (keys) { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= blobs_with_metadata->size()) + if (current_index >= keys->size()) { is_finished = true; return {}; } - return (*blobs_with_metadata)[current_index]; + auto key = (*keys)[current_index]; + ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); + return {key, object_metadata}; } else { @@ -946,11 +933,12 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() const auto & idxs = typeid_cast(*block.getByName("_idx").column); std::lock_guard lock(next_mutex); - blob_path_with_globs.reset(); - blob_path_with_globs.emplace(); + blobs_with_metadata.reset(); + blobs_with_metadata.emplace(); for (UInt64 idx : idxs.getData()) { - total_size.fetch_add(new_batch[idx].metadata.size_bytes, std::memory_order_relaxed); + if (file_progress_callback) + file_progress_callback(FileProgress(0, new_batch[idx].metadata.size_bytes)); blobs_with_metadata->emplace_back(std::move(new_batch[idx])); if (outer_blobs) outer_blobs->emplace_back(blobs_with_metadata->back()); @@ -963,8 +951,11 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); - for (const auto & [_, info] : *blobs_with_metadata) - total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); + if (file_progress_callback) + { + for (const auto & [_, info] : *blobs_with_metadata) + file_progress_callback(FileProgress(0, info.size_bytes)); + } } } @@ -1011,17 +1002,9 @@ Chunk StorageAzureBlobSource::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); + progress(num_rows, 0); const auto & file_path = reader.getPath(); - if (num_rows && total_objects_size) - { - size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } - for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -1046,13 +1029,6 @@ Chunk StorageAzureBlobSource::generate() if (!reader) break; - size_t object_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0); - /// Adjust total_rows_approx_accumulated with new total size. - if (total_objects_size) - total_rows_approx_accumulated = static_cast( - std::ceil(static_cast(total_objects_size + object_size) / total_objects_size * total_rows_approx_accumulated)); - total_objects_size += object_size; - /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. create_reader_pool.wait(); @@ -1083,7 +1059,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( AzureObjectStorage * object_storage_, const String & container_, std::shared_ptr file_iterator_) - :ISource(getHeader(sample_block_, requested_virtual_columns_)) + :ISource(getHeader(sample_block_, requested_virtual_columns_), false) , WithContext(context_) , requested_virtual_columns(requested_virtual_columns_) , format(format_) @@ -1101,13 +1077,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( { reader = createReader(); if (reader) - { - const auto & read_buf = reader.getReadBuffer(); - if (read_buf) - total_objects_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0); - reader_future = createReaderAsync(); - } } @@ -1149,7 +1119,7 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(container) / current_key, std::move(read_buf), input_format, std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(container) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageAzureBlobSource::createReaderAsync() @@ -1163,6 +1133,7 @@ std::unique_ptr StorageAzureBlobSource::createAzureReadBuffer(const read_settings.enable_filesystem_cache = false; auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; const bool object_too_small = object_size <= 2 * download_buffer_size; + object_storage->setProgressCallback(getContext()); // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 25c791f17004..a78ba691b571 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -153,7 +153,8 @@ class StorageAzureBlobSource : public ISource, WithContext ASTPtr query_, const Block & virtual_header_, ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_); + RelativePathsWithMetadata * outer_blobs_, + std::function file_progress_callback_ = {}); RelativePathWithMetadata next(); size_t getTotalSize() const; @@ -182,6 +183,8 @@ class StorageAzureBlobSource : public ISource, WithContext std::atomic is_finished = false; std::atomic is_initialized = false; std::mutex next_mutex; + + std::function file_progress_callback; }; StorageAzureBlobSource( @@ -225,12 +228,10 @@ class StorageAzureBlobSource : public ISource, WithContext ReaderHolder( String path_, std::unique_ptr read_buf_, - std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) : path(std::move(path_)) , read_buf(std::move(read_buf_)) - , input_format(input_format_) , pipeline(std::move(pipeline_)) , reader(std::move(reader_)) { @@ -251,7 +252,6 @@ class StorageAzureBlobSource : public ISource, WithContext /// reader uses pipeline, pipeline uses read_buf. reader = std::move(other.reader); pipeline = std::move(other.pipeline); - input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); path = std::move(other.path); return *this; @@ -262,14 +262,9 @@ class StorageAzureBlobSource : public ISource, WithContext const PullingPipelineExecutor * operator->() const { return reader.get(); } const String & getPath() const { return path; } - const std::unique_ptr & getReadBuffer() const { return read_buf; } - - const std::shared_ptr & getFormat() const { return input_format; } - private: String path; std::unique_ptr read_buf; - std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; }; @@ -282,11 +277,6 @@ class StorageAzureBlobSource : public ISource, WithContext ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; - size_t total_objects_size = 0; - /// Recreate ReadBuffer and Pipeline for each file. ReaderHolder createReader(); std::future createReaderAsync(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7c04de1a28ab..f196415e2dcb 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -295,13 +295,8 @@ std::unique_ptr createReadBuffer( std::unique_ptr nested_buffer = selectReadBuffer(current_path, use_table_fd, table_fd, file_stat, context); - /// For clickhouse-local and clickhouse-client add progress callback to display progress bar. - if (context->getApplicationType() == Context::ApplicationType::LOCAL - || context->getApplicationType() == Context::ApplicationType::CLIENT) - { - auto & in = static_cast(*nested_buffer); - in.setProgressCallback(context); - } + auto & in = static_cast(*nested_buffer); + in.setProgressCallback(context); int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method, zstd_window_log_max); @@ -607,7 +602,7 @@ class StorageFileSource : public ISource ColumnsDescription columns_description_, const Block & block_for_format_, std::unique_ptr read_buf_) - : ISource(getBlockForSource(block_for_format_, files_info_)) + : ISource(getBlockForSource(block_for_format_, files_info_), false) , storage(std::move(storage_)) , storage_snapshot(storage_snapshot_) , files_info(std::move(files_info_)) @@ -722,12 +717,6 @@ class StorageFileSource : public ISource read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, context); } - size_t file_size = tryGetFileSizeFromReadBuffer(*read_buf).value_or(0); - /// Adjust total_rows_approx_accumulated with new total size. - if (total_files_size) - total_rows_approx_accumulated = static_cast(std::ceil(static_cast(total_files_size + file_size) / total_files_size * total_rows_approx_accumulated)); - total_files_size += file_size; - const Settings & settings = context->getSettingsRef(); chassert(!storage->paths.empty()); const auto max_parsing_threads = std::max(settings.max_threads/ storage->paths.size(), 1UL); @@ -753,6 +742,7 @@ class StorageFileSource : public ISource if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); + progress(num_rows, 0); /// Enrich with virtual columns. if (files_info->need_path_column) @@ -770,14 +760,6 @@ class StorageFileSource : public ISource chunk.addColumn(column->convertToFullColumnIfConst()); } - if (num_rows && total_files_size) - { - size_t chunk_size = input_format->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } return chunk; } @@ -816,12 +798,6 @@ class StorageFileSource : public ISource bool finished_generate = false; std::shared_lock shared_lock; - - UInt64 total_rows_approx_accumulated = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_max = 0; - - size_t total_files_size = 0; }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f4791e45e2b4..d933ffe8041f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -148,7 +148,8 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext const Block & virtual_header_, ContextPtr context_, KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_) + const S3Settings::RequestSettings & request_settings_, + std::function progress_callback_) : WithContext(context_) , client(S3::Client::create(client_)) , globbed_uri(globbed_uri_) @@ -158,6 +159,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext , request_settings(request_settings_) , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) + , progress_callback(progress_callback_) { if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); @@ -194,11 +196,6 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext return nextAssumeLocked(); } - size_t getTotalSize() const - { - return total_size.load(std::memory_order_relaxed); - } - ~Impl() { list_objects_pool.wait(); @@ -312,15 +309,19 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext buffer.reserve(block.rows()); for (UInt64 idx : idxs.getData()) { - total_size.fetch_add(temp_buffer[idx].info->size, std::memory_order_relaxed); + if (progress_callback) + progress_callback(FileProgress(0, temp_buffer[idx].info->size)); buffer.emplace_back(std::move(temp_buffer[idx])); } } else { buffer = std::move(temp_buffer); - for (const auto & [_, info] : buffer) - total_size.fetch_add(info->size, std::memory_order_relaxed); + if (progress_callback) + { + for (const auto & [_, info] : buffer) + progress_callback(FileProgress(0, info->size)); + } } /// Set iterator only after the whole batch is processed @@ -381,7 +382,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; - std::atomic total_size = 0; + std::function progress_callback; }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( @@ -391,8 +392,9 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( const Block & virtual_header, ContextPtr context, KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) + const S3Settings::RequestSettings & request_settings_, + std::function progress_callback_) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_, progress_callback_)) { } @@ -401,11 +403,6 @@ StorageS3Source::KeyWithInfo StorageS3Source::DisclosedGlobIterator::next() return pimpl->next(); } -size_t StorageS3Source::DisclosedGlobIterator::getTotalSize() const -{ - return pimpl->getTotalSize(); -} - class StorageS3Source::KeysIterator::Impl : WithContext { public: @@ -418,23 +415,26 @@ class StorageS3Source::KeysIterator::Impl : WithContext ASTPtr query_, const Block & virtual_header_, ContextPtr context_, - bool need_total_size, - KeysWithInfo * read_keys_) + KeysWithInfo * read_keys_, + std::function progress_callback_) : WithContext(context_) + , keys(keys_) + , client(S3::Client::create(client_)) + , version_id(version_id_) , bucket(bucket_) + , request_settings(request_settings_) , query(query_) , virtual_header(virtual_header_) + , progress_callback(progress_callback_) { - Strings all_keys = keys_; - /// Create a virtual block with one row to construct filter - if (query && virtual_header && !all_keys.empty()) + if (query && virtual_header && !keys.empty()) { /// Append "idx" column as the filter result virtual_header.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); auto block = virtual_header.cloneEmpty(); - addPathToVirtualColumns(block, fs::path(bucket) / all_keys.front(), 0); + addPathToVirtualColumns(block, fs::path(bucket) / keys.front(), 0); ASTPtr filter_ast; VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast); @@ -442,8 +442,8 @@ class StorageS3Source::KeysIterator::Impl : WithContext if (filter_ast) { block = virtual_header.cloneEmpty(); - for (size_t i = 0; i < all_keys.size(); ++i) - addPathToVirtualColumns(block, fs::path(bucket) / all_keys[i], i); + for (size_t i = 0; i < keys.size(); ++i) + addPathToVirtualColumns(block, fs::path(bucket) / keys[i], i); VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); @@ -451,29 +451,17 @@ class StorageS3Source::KeysIterator::Impl : WithContext Strings filtered_keys; filtered_keys.reserve(block.rows()); for (UInt64 idx : idxs.getData()) - filtered_keys.emplace_back(std::move(all_keys[idx])); + filtered_keys.emplace_back(std::move(keys[idx])); - all_keys = std::move(filtered_keys); + keys = std::move(filtered_keys); } } - for (auto && key : all_keys) + if (read_keys_) { - std::optional info; - /// In case all_keys.size() > 1, avoid getting object info now - /// (it will be done anyway eventually, but with delay and in parallel). - /// But progress bar will not work in this case. - if (need_total_size && all_keys.size() == 1) - { - info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_); - total_size += info->size; - } - - keys.emplace_back(std::move(key), std::move(info)); + for (const auto & key : keys) + read_keys_->push_back({key, {}}); } - - if (read_keys_) - *read_keys_ = keys; } KeyWithInfo next() @@ -481,24 +469,27 @@ class StorageS3Source::KeysIterator::Impl : WithContext size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= keys.size()) return {}; + auto key = keys[current_index]; + std::optional info; + if (progress_callback) + { + info = S3::getObjectInfo(*client, bucket, key, version_id, request_settings); + progress_callback(FileProgress(0, info->size)); + } - return keys[current_index]; - } - - size_t getTotalSize() const - { - return total_size; + return {key, info}; } private: - KeysWithInfo keys; + Strings keys; std::atomic_size_t index = 0; - + std::unique_ptr client; + String version_id; String bucket; + S3Settings::RequestSettings request_settings; ASTPtr query; Block virtual_header; - - size_t total_size = 0; + std::function progress_callback; }; StorageS3Source::KeysIterator::KeysIterator( @@ -510,11 +501,11 @@ StorageS3Source::KeysIterator::KeysIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, - bool need_total_size, - KeysWithInfo * read_keys) + KeysWithInfo * read_keys, + std::function progress_callback_) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_header, context, need_total_size, read_keys)) + query, virtual_header, context, read_keys, progress_callback_)) { } @@ -523,11 +514,6 @@ StorageS3Source::KeyWithInfo StorageS3Source::KeysIterator::next() return pimpl->next(); } -size_t StorageS3Source::KeysIterator::getTotalSize() const -{ - return pimpl->getTotalSize(); -} - Block StorageS3Source::getHeader(Block sample_block, const std::vector & requested_virtual_columns) { for (const auto & virtual_column : requested_virtual_columns) @@ -552,7 +538,7 @@ StorageS3Source::StorageS3Source( const String & version_id_, std::shared_ptr file_iterator_, const size_t download_thread_num_) - : ISource(getHeader(sample_block_, requested_virtual_columns_)) + : ISource(getHeader(sample_block_, requested_virtual_columns_), false) , WithContext(context_) , name(std::move(name_)) , bucket(bucket_) @@ -573,10 +559,7 @@ StorageS3Source::StorageS3Source( { reader = createReader(); if (reader) - { - total_objects_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0); reader_future = createReaderAsync(); - } } StorageS3Source::ReaderHolder StorageS3Source::createReader() @@ -614,7 +597,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), input_format, std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -638,10 +621,13 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k return createAsyncS3ReadBuffer(key, read_settings, object_size); } - return std::make_unique( + auto buf = std::make_unique( client, bucket, key, version_id, request_settings, read_settings, /*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0, /*restricted_seek_*/ false, object_size); + + buf->setProgressCallback(getContext()); + return buf; } std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( @@ -652,7 +638,7 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( [this, read_settings, object_size] (const std::string & path, size_t read_until_position) -> std::unique_ptr { - return std::make_unique( + auto buf = std::make_unique( client, bucket, path, @@ -664,6 +650,8 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( read_until_position, /* restricted_seek */true, object_size); + buf->setProgressCallback(getContext()); + return buf; }; auto s3_impl = std::make_unique( @@ -713,17 +701,10 @@ Chunk StorageS3Source::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); + progress(num_rows, 0); const auto & file_path = reader.getPath(); - if (num_rows && total_objects_size) - { - size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } - for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") @@ -748,13 +729,6 @@ Chunk StorageS3Source::generate() if (!reader) break; - size_t object_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0); - /// Adjust total_rows_approx_accumulated with new total size. - if (total_objects_size) - total_rows_approx_accumulated = static_cast( - std::ceil(static_cast(total_objects_size + object_size) / total_objects_size * total_rows_approx_accumulated)); - total_objects_size += object_size; - /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. create_reader_pool.wait(); @@ -990,8 +964,8 @@ std::shared_ptr StorageS3::createFileIterator( ContextPtr local_context, ASTPtr query, const Block & virtual_block, - bool need_total_size, - KeysWithInfo * read_keys) + KeysWithInfo * read_keys, + std::function progress_callback) { if (distributed_processing) { @@ -1002,14 +976,14 @@ std::shared_ptr StorageS3::createFileIterator( /// Iterate through disclosed globs and make a source for each file return std::make_shared( *configuration.client, configuration.url, query, virtual_block, - local_context, read_keys, configuration.request_settings); + local_context, read_keys, configuration.request_settings, progress_callback); } else { return std::make_shared( *configuration.client, configuration.url.version_id, configuration.keys, configuration.url.bucket, configuration.request_settings, query, - virtual_block, local_context, need_total_size, read_keys); + virtual_block, local_context, read_keys, progress_callback); } } @@ -1059,7 +1033,7 @@ Pipe StorageS3::read( } std::shared_ptr iterator_wrapper = createFileIterator( - query_configuration, distributed_processing, local_context, query_info.query, virtual_block); + query_configuration, distributed_processing, local_context, query_info.query, virtual_block, nullptr, local_context->getFileProgressCallback()); ColumnsDescription columns_description; Block block_for_format; @@ -1459,7 +1433,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( { KeysWithInfo read_keys; - auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, false, &read_keys); + auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, &read_keys); std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 130538336230..16d075a67d2d 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -56,7 +56,6 @@ class StorageS3Source : public ISource, WithContext public: virtual ~IIterator() = default; virtual KeyWithInfo next() = 0; - virtual size_t getTotalSize() const = 0; KeyWithInfo operator ()() { return next(); } }; @@ -71,10 +70,10 @@ class StorageS3Source : public ISource, WithContext const Block & virtual_header, ContextPtr context, KeysWithInfo * read_keys_ = nullptr, - const S3Settings::RequestSettings & request_settings_ = {}); + const S3Settings::RequestSettings & request_settings_ = {}, + std::function progress_callback_ = {}); KeyWithInfo next() override; - size_t getTotalSize() const override; private: class Impl; @@ -94,11 +93,10 @@ class StorageS3Source : public ISource, WithContext ASTPtr query, const Block & virtual_header, ContextPtr context, - bool need_total_size = true, - KeysWithInfo * read_keys = nullptr); + KeysWithInfo * read_keys = nullptr, + std::function progress_callback_ = {}); KeyWithInfo next() override; - size_t getTotalSize() const override; private: class Impl; @@ -113,8 +111,6 @@ class StorageS3Source : public ISource, WithContext KeyWithInfo next() override { return {callback(), {}}; } - size_t getTotalSize() const override { return 0; } - private: ReadTaskCallback callback; }; @@ -163,12 +159,10 @@ class StorageS3Source : public ISource, WithContext ReaderHolder( String path_, std::unique_ptr read_buf_, - std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) : path(std::move(path_)) , read_buf(std::move(read_buf_)) - , input_format(input_format_) , pipeline(std::move(pipeline_)) , reader(std::move(reader_)) { @@ -189,16 +183,11 @@ class StorageS3Source : public ISource, WithContext /// reader uses pipeline, pipeline uses read_buf. reader = std::move(other.reader); pipeline = std::move(other.pipeline); - input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); path = std::move(other.path); return *this; } - const std::unique_ptr & getReadBuffer() const { return read_buf; } - - const std::shared_ptr & getFormat() const { return input_format; } - explicit operator bool() const { return reader != nullptr; } PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } @@ -207,7 +196,6 @@ class StorageS3Source : public ISource, WithContext private: String path; std::unique_ptr read_buf; - std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; }; @@ -224,11 +212,6 @@ class StorageS3Source : public ISource, WithContext ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; - size_t total_objects_size = 0; - /// Recreate ReadBuffer and Pipeline for each file. ReaderHolder createReader(); std::future createReaderAsync(); @@ -352,8 +335,8 @@ class StorageS3 : public IStorage ContextPtr local_context, ASTPtr query, const Block & virtual_block, - bool need_total_size = true, - KeysWithInfo * read_keys = nullptr); + KeysWithInfo * read_keys = nullptr, + std::function progress_callback = {}); static ColumnsDescription getTableStructureFromDataImpl( const Configuration & configuration, diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 153a3b7f11bb..18ae44bc1ad9 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -86,7 +86,7 @@ void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const { auto iterator = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_block, context); + *s3_configuration.client, s3_configuration.url, query, virtual_block, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next().key; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1d6aed204cba..1ea0eb5a88ee 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -234,7 +234,7 @@ StorageURLSource::StorageURLSource( const HTTPHeaderEntries & headers_, const URIParams & params, bool glob_url) - : ISource(getHeader(sample_block, requested_virtual_columns_)), name(std::move(name_)), requested_virtual_columns(requested_virtual_columns_), uri_iterator(uri_iterator_) + : ISource(getHeader(sample_block, requested_virtual_columns_), false), name(std::move(name_)), requested_virtual_columns(requested_virtual_columns_), uri_iterator(uri_iterator_) { auto headers = getHeaders(headers_); @@ -261,7 +261,8 @@ StorageURLSource::StorageURLSource( credentials, headers, glob_url, - current_uri_options.size() == 1); + current_uri_options.size() == 1, + context->getFileProgressCallback()); /// If file is empty and engine_url_skip_empty_files=1, skip it and go to the next file. } @@ -270,22 +271,11 @@ StorageURLSource::StorageURLSource( curr_uri = uri_and_buf.first; read_buf = std::move(uri_and_buf.second); - size_t file_size = 0; - try + if (auto progress_callback = context->getFileProgressCallback()) { - file_size = getFileSizeFromReadBuffer(*read_buf); - } - catch (...) - { - // we simply continue without updating total_size - } - - if (file_size) - { - /// Adjust total_rows_approx_accumulated with new total size. - if (total_size) - total_rows_approx_accumulated = static_cast(std::ceil(static_cast(total_size + file_size) / total_size * total_rows_approx_accumulated)); - total_size += file_size; + size_t file_size = tryGetFileSizeFromReadBuffer(*read_buf).value_or(0); + LOG_DEBUG(&Poco::Logger::get("URL"), "Send file size {}", file_size); + progress_callback(FileProgress(0, file_size)); } // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. @@ -331,14 +321,7 @@ Chunk StorageURLSource::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - if (num_rows && total_size) - { - size_t chunk_size = input_format->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } + progress(num_rows, 0); const String & path{curr_uri.getPath()}; @@ -376,7 +359,8 @@ std::pair> StorageURLSource: Poco::Net::HTTPBasicCredentials & credentials, const HTTPHeaderEntries & headers, bool glob_url, - bool delay_initialization) + bool delay_initialization, + std::function file_progress_callback) { String first_exception_message; ReadSettings read_settings = context->getReadSettings(); @@ -418,6 +402,7 @@ std::pair> StorageURLSource: continue; } + res->setProgressCallback(file_progress_callback); return std::make_tuple(request_uri, std::move(res)); } catch (...) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index e3305cda89e0..315a5f9897bc 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -195,7 +195,8 @@ class StorageURLSource : public ISource Poco::Net::HTTPBasicCredentials & credentials, const HTTPHeaderEntries & headers, bool glob_url, - bool delay_initialization); + bool delay_initialization, + std::function file_progress_callback = {}); private: using InitializeFunc = std::function; @@ -212,11 +213,6 @@ class StorageURLSource : public ISource std::unique_ptr reader; Poco::Net::HTTPBasicCredentials credentials; - - size_t total_size = 0; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; }; class StorageURLSink : public SinkToStorage From f48cd0f926338d2420b123aaed3bc22fbc2969e9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Jun 2023 17:30:24 +0000 Subject: [PATCH 047/226] Delete updateRowsProgressApprox implementation --- src/Storages/ReadFromStorageProgress.cpp | 52 ------------------------ src/Storages/ReadFromStorageProgress.h | 18 -------- 2 files changed, 70 deletions(-) delete mode 100644 src/Storages/ReadFromStorageProgress.cpp delete mode 100644 src/Storages/ReadFromStorageProgress.h diff --git a/src/Storages/ReadFromStorageProgress.cpp b/src/Storages/ReadFromStorageProgress.cpp deleted file mode 100644 index 8ad1cf922095..000000000000 --- a/src/Storages/ReadFromStorageProgress.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -void updateRowsProgressApprox( - ISource & source, - size_t num_rows, - UInt64 chunk_bytes_size, - UInt64 total_result_size, - UInt64 & total_rows_approx_accumulated, - size_t & total_rows_count_times, - UInt64 & total_rows_approx_max) -{ - if (!total_result_size) - return; - - if (!num_rows) - return; - - const auto progress = source.getReadProgress(); - if (progress && !progress->limits.empty()) - { - for (const auto & limit : progress->limits) - { - if (limit.leaf_limits.max_rows || limit.leaf_limits.max_bytes - || limit.local_limits.size_limits.max_rows || limit.local_limits.size_limits.max_bytes) - return; - } - } - - const auto bytes_per_row = std::ceil(static_cast(chunk_bytes_size) / num_rows); - size_t total_rows_approx = static_cast(std::ceil(static_cast(total_result_size) / bytes_per_row)); - total_rows_approx_accumulated += total_rows_approx; - ++total_rows_count_times; - total_rows_approx = total_rows_approx_accumulated / total_rows_count_times; - - /// We need to add diff, because total_rows_approx is incremental value. - /// It would be more correct to send total_rows_approx as is (not a diff), - /// but incrementation of total_rows_to_read does not allow that. - /// A new counter can be introduced for that to be sent to client, but it does not worth it. - if (total_rows_approx > total_rows_approx_max) - { - size_t diff = total_rows_approx - total_rows_approx_max; - source.addTotalRowsApprox(diff); - total_rows_approx_max = total_rows_approx; - } -} - -} diff --git a/src/Storages/ReadFromStorageProgress.h b/src/Storages/ReadFromStorageProgress.h deleted file mode 100644 index 2be37d26feea..000000000000 --- a/src/Storages/ReadFromStorageProgress.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class ISource; - -void updateRowsProgressApprox( - ISource & source, - size_t num_rows, - UInt64 chunk_bytes_size, - UInt64 total_result_size, - UInt64 & total_rows_approx_accumulated, - size_t & total_rows_count_times, - UInt64 & total_rows_approx_max); - -} From 24fab7bfde4557303335609949548632dbafc218 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Jun 2023 18:48:15 +0000 Subject: [PATCH 048/226] Remove old includes --- src/Storages/HDFS/StorageHDFS.cpp | 1 - src/Storages/StorageAzureBlob.cpp | 1 - src/Storages/StorageFile.cpp | 1 - src/Storages/StorageS3.cpp | 1 - src/Storages/StorageURL.cpp | 1 - 5 files changed, 5 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 79cda3050d6e..c6f0bd3f18d2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 8e06ceda8858..1af7afc952fb 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f196415e2dcb..914fc4329079 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d933ffe8041f..5a75da7a1880 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1ea0eb5a88ee..5a8f94d07fda 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include From c679dd400e7119c087ed2dedb58dabdfdd23a2d3 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Jun 2023 13:43:40 +0000 Subject: [PATCH 049/226] Make better --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 8 +-- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 5 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 10 +--- .../AzureBlobStorage/AzureObjectStorage.h | 5 -- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 + src/IO/ReadBufferFromFileBase.h | 2 +- src/IO/ReadBufferFromS3.cpp | 13 +---- src/IO/ReadBufferFromS3.h | 6 +-- src/IO/ReadWriteBufferFromHTTP.cpp | 14 +---- src/IO/ReadWriteBufferFromHTTP.h | 5 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 9 ---- src/Storages/HDFS/ReadBufferFromHDFS.h | 3 -- src/Storages/HDFS/StorageHDFS.cpp | 6 ++- src/Storages/HDFS/StorageHDFS.h | 1 + src/Storages/StorageAzureBlob.cpp | 6 +-- src/Storages/StorageAzureBlob.h | 5 ++ src/Storages/StorageFile.cpp | 6 +-- src/Storages/StorageS3.cpp | 52 +++++++++---------- src/Storages/StorageS3.h | 6 +++ src/Storages/StorageURL.cpp | 16 +++--- src/Storages/StorageURL.h | 3 +- 21 files changed, 67 insertions(+), 116 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 6a328de03418..129bb97be097 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace ProfileEvents @@ -37,8 +36,7 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( size_t max_single_download_retries_, bool use_external_buffer_, bool restricted_seek_, - size_t read_until_position_, - std::function progress_callback_) + size_t read_until_position_) : ReadBufferFromFileBase(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size, nullptr, 0) , blob_container_client(blob_container_client_) , path(path_) @@ -49,7 +47,6 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( , use_external_buffer(use_external_buffer_) , restricted_seek(restricted_seek_) , read_until_position(read_until_position_) - , progress_callback(progress_callback_) { if (!use_external_buffer) { @@ -130,9 +127,6 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() if (bytes_read == 0) return false; - if (progress_callback) - progress_callback(FileProgress(bytes_read)); - BufferBase::set(data_ptr, bytes_read, 0); offset += bytes_read; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 6f683dcf1cec..4e21f5436536 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -25,8 +25,7 @@ class ReadBufferFromAzureBlobStorage : public ReadBufferFromFileBase size_t max_single_download_retries_, bool use_external_buffer_ = false, bool restricted_seek_ = false, - size_t read_until_position_ = 0, - std::function progress_callback_ = {}); + size_t read_until_position_ = 0); off_t seek(off_t off, int whence) override; @@ -75,8 +74,6 @@ class ReadBufferFromAzureBlobStorage : public ReadBufferFromFileBase size_t data_capacity; Poco::Logger * log = &Poco::Logger::get("ReadBufferFromAzureBlobStorage"); - - std::function progress_callback; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 982c376404aa..dbb418510532 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -191,7 +191,7 @@ std::unique_ptr AzureObjectStorage::readObject( /// NOLI return std::make_unique( client.get(), object.remote_path, patchSettings(read_settings), settings_ptr->max_single_read_retries, - settings_ptr->max_single_download_retries, false, false, 0, progress_callback); + settings_ptr->max_single_download_retries); } std::unique_ptr AzureObjectStorage::readObjects( /// NOLINT @@ -216,8 +216,7 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL settings_ptr->max_single_download_retries, /* use_external_buffer */true, /* restricted_seek */true, - read_until_position, - progress_callback); + read_until_position); }; switch (read_settings.remote_fs_method) @@ -391,11 +390,6 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std ); } -void AzureObjectStorage::setProgressCallback(const ContextPtr & context) -{ - progress_callback = context->getFileProgressCallback(); -} - } #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index ee144cdd56ec..b5f81cef2350 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -123,9 +123,6 @@ class AzureObjectStorage : public IObjectStorage bool isRemote() const override { return true; } - /// Set progress callback to read buffer while reading from storage. - void setProgressCallback(const ContextPtr & context); - private: const String name; /// client used to access the files in the Blob Storage cloud @@ -135,8 +132,6 @@ class AzureObjectStorage : public IObjectStorage Poco::Logger * log; DataSourceDescription data_source_description; - - std::function progress_callback; }; } diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 9b1c132cc010..46e8d914cba8 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -99,6 +99,8 @@ size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t o n = std::min(n, mapped.getLength() - offset); memcpy(to, mapped.getData() + offset, n); + if (profile_callback) + profile_callback(ProfileInfo{.bytes_read = n}); return n; } diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index 2abdf883ab09..b77db29bc238 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -52,7 +52,7 @@ class ReadBufferFromFileBase : public BufferWithOwnMemory, p size_t getFileSize() override; - virtual void setProgressCallback(ContextPtr context); + void setProgressCallback(ContextPtr context); protected: std::optional file_size; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 36ff81a85d42..2f29a7650832 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -163,13 +163,11 @@ bool ReadBufferFromS3::nextImpl() offset += working_buffer.size(); if (read_settings.remote_throttler) read_settings.remote_throttler->add(working_buffer.size(), ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - if (progress_callback) - progress_callback(FileProgress(working_buffer.size())); return true; } -size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & custom_progress_callback) +size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) { if (n == 0) return 0; @@ -186,9 +184,7 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons auto result = sendRequest(range_begin, range_begin + n - 1); std::istream & istr = result.GetBody(); - size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, custom_progress_callback); - if (progress_callback) - progress_callback(FileProgress(bytes, 0)); + size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, progress_callback); ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes); @@ -419,11 +415,6 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t range_begin } } -void ReadBufferFromS3::setProgressCallback(DB::ContextPtr context) -{ - progress_callback = context->getFileProgressCallback(); -} - } #endif diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 824038c7af01..0f665861a1ea 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -77,12 +77,10 @@ class ReadBufferFromS3 : public ReadBufferFromFileBase String getFileName() const override { return bucket + "/" + key; } - size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & custom_progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) override; bool supportsReadAt() override { return true; } - void setProgressCallback(ContextPtr context) override; - private: std::unique_ptr initialize(); @@ -102,8 +100,6 @@ class ReadBufferFromS3 : public ReadBufferFromFileBase /// There is different seek policy for disk seek and for non-disk seek /// (non-disk seek is applied for seekable input formats: orc, arrow, parquet). bool restricted_seek; - - std::function progress_callback; }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 7bd7f4a9b8ed..f38c1bcaacc8 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -587,13 +587,11 @@ bool ReadWriteBufferFromHTTPBase::nextImpl() internal_buffer = impl->buffer(); working_buffer = internal_buffer; offset_from_begin_pos += working_buffer.size(); - if (progress_callback) - progress_callback(FileProgress(working_buffer.size())); return true; } template -size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & custom_progress_callback) +size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) { /// Caller must have checked supportsReadAt(). /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. @@ -635,9 +633,7 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si toString(response.getStatus()), uri_.toString(), offset, offset + n); bool cancelled; - size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, custom_progress_callback, &cancelled); - if (progress_callback) - progress_callback(FileProgress(r)); + size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); return r; } catch (const Poco::Exception & e) @@ -783,12 +779,6 @@ void ReadWriteBufferFromHTTPBase::setNextCallback(NextCallb next_callback(count()); } -template -void ReadWriteBufferFromHTTPBase::setProgressCallback(std::function file_progress_callback_) -{ - progress_callback = file_progress_callback_; -} - template const std::string & ReadWriteBufferFromHTTPBase::getCompressionMethod() const { return content_encoding; } diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 18bd31fcdce2..2d2ae5fe724e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -92,7 +92,6 @@ namespace detail HTTPHeaderEntries http_header_entries; const RemoteHostFilter * remote_host_filter = nullptr; std::function next_callback; - std::function progress_callback; size_t buffer_size; bool use_external_buffer; @@ -177,7 +176,7 @@ namespace detail bool nextImpl() override; - size_t readBigAt(char * to, size_t n, size_t offset, const std::function & custom_progress_callback) override; + size_t readBigAt(char * to, size_t n, size_t offset, const std::function & progress_callback) override; off_t getPosition() override; @@ -200,8 +199,6 @@ namespace detail /// passed through the buffer void setNextCallback(NextCallback next_callback_); - void setProgressCallback(std::function progress_callback_); - const std::string & getCompressionMethod() const; std::optional getLastModificationTime(); diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 2c2c5047cb15..7697776297e8 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -198,11 +198,7 @@ bool ReadBufferFromHDFS::nextImpl() auto result = impl->next(); if (result) - { BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); /// use the buffer returned by `impl` - if (progress_callback) - progress_callback(FileProgress(working_buffer.size())); - } return result; } @@ -262,11 +258,6 @@ String ReadBufferFromHDFS::getFileName() const return impl->hdfs_file_path; } -void ReadBufferFromHDFS::setProgressCallback(DB::ContextPtr context) -{ - progress_callback = context->getFileProgressCallback(); -} - } #endif diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 3dce6a93cbac..64adbc62789e 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -48,12 +48,9 @@ struct ReadBufferFromHDFSImpl; String getFileName() const override; - void setProgressCallback(ContextPtr context) override; - private: std::unique_ptr impl; bool use_external_buffer; - std::function progress_callback; }; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c6f0bd3f18d2..15632f96bc1c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -400,7 +400,7 @@ bool HDFSSource::initialize() current_path = path_with_info.path; - auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); + input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -437,7 +437,8 @@ Chunk HDFSSource::generate() { Columns columns = chunk.getColumns(); UInt64 num_rows = chunk.getNumRows(); - progress(num_rows, 0); + size_t chunk_size = input_format->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); for (const auto & virtual_column : requested_virtual_columns) { @@ -461,6 +462,7 @@ Chunk HDFSSource::generate() reader.reset(); pipeline.reset(); + input_format.reset(); read_buf.reset(); if (!initialize()) diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 5a3b97a0e3c8..c487c9df20d4 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -164,6 +164,7 @@ class HDFSSource : public ISource, WithContext ColumnsDescription columns_description; std::unique_ptr read_buf; + std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; String current_path; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 1af7afc952fb..760c84168277 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1001,7 +1001,8 @@ Chunk StorageAzureBlobSource::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - progress(num_rows, 0); + size_t chunk_size = reader.getInputFormat()->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); const auto & file_path = reader.getPath(); for (const auto & virtual_column : requested_virtual_columns) @@ -1118,7 +1119,7 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(container) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(container) / current_key, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; } std::future StorageAzureBlobSource::createReaderAsync() @@ -1132,7 +1133,6 @@ std::unique_ptr StorageAzureBlobSource::createAzureReadBuffer(const read_settings.enable_filesystem_cache = false; auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; const bool object_too_small = object_size <= 2 * download_buffer_size; - object_storage->setProgressCallback(getContext()); // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index a78ba691b571..392923c5178c 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -228,10 +228,12 @@ class StorageAzureBlobSource : public ISource, WithContext ReaderHolder( String path_, std::unique_ptr read_buf_, + std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) : path(std::move(path_)) , read_buf(std::move(read_buf_)) + , input_format(std::move(input_format_)) , pipeline(std::move(pipeline_)) , reader(std::move(reader_)) { @@ -252,6 +254,7 @@ class StorageAzureBlobSource : public ISource, WithContext /// reader uses pipeline, pipeline uses read_buf. reader = std::move(other.reader); pipeline = std::move(other.pipeline); + input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); path = std::move(other.path); return *this; @@ -261,10 +264,12 @@ class StorageAzureBlobSource : public ISource, WithContext PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } const String & getPath() const { return path; } + const IInputFormat * getInputFormat() const { return input_format.get(); } private: String path; std::unique_ptr read_buf; + std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 914fc4329079..8eecfb684c5e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -294,9 +294,6 @@ std::unique_ptr createReadBuffer( std::unique_ptr nested_buffer = selectReadBuffer(current_path, use_table_fd, table_fd, file_stat, context); - auto & in = static_cast(*nested_buffer); - in.setProgressCallback(context); - int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method, zstd_window_log_max); } @@ -741,7 +738,8 @@ class StorageFileSource : public ISource if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - progress(num_rows, 0); + size_t chunk_size = input_format->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size); /// Enrich with virtual columns. if (files_info->need_path_column) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 5a75da7a1880..996cf315cc3b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -148,7 +148,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext ContextPtr context_, KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_, - std::function progress_callback_) + std::function file_progress_callback_) : WithContext(context_) , client(S3::Client::create(client_)) , globbed_uri(globbed_uri_) @@ -158,7 +158,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext , request_settings(request_settings_) , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) - , progress_callback(progress_callback_) + , file_progress_callback(file_progress_callback_) { if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); @@ -308,18 +308,18 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext buffer.reserve(block.rows()); for (UInt64 idx : idxs.getData()) { - if (progress_callback) - progress_callback(FileProgress(0, temp_buffer[idx].info->size)); + if (file_progress_callback) + file_progress_callback(FileProgress(0, temp_buffer[idx].info->size)); buffer.emplace_back(std::move(temp_buffer[idx])); } } else { buffer = std::move(temp_buffer); - if (progress_callback) + if (file_progress_callback) { for (const auto & [_, info] : buffer) - progress_callback(FileProgress(0, info->size)); + file_progress_callback(FileProgress(0, info->size)); } } @@ -381,7 +381,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; - std::function progress_callback; + std::function file_progress_callback; }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( @@ -392,8 +392,8 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( ContextPtr context, KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_, - std::function progress_callback_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_, progress_callback_)) + std::function file_progress_callback_) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_, file_progress_callback_)) { } @@ -415,7 +415,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext const Block & virtual_header_, ContextPtr context_, KeysWithInfo * read_keys_, - std::function progress_callback_) + std::function file_progress_callback_) : WithContext(context_) , keys(keys_) , client(S3::Client::create(client_)) @@ -424,7 +424,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext , request_settings(request_settings_) , query(query_) , virtual_header(virtual_header_) - , progress_callback(progress_callback_) + , file_progress_callback(file_progress_callback_) { /// Create a virtual block with one row to construct filter if (query && virtual_header && !keys.empty()) @@ -470,10 +470,10 @@ class StorageS3Source::KeysIterator::Impl : WithContext return {}; auto key = keys[current_index]; std::optional info; - if (progress_callback) + if (file_progress_callback) { info = S3::getObjectInfo(*client, bucket, key, version_id, request_settings); - progress_callback(FileProgress(0, info->size)); + file_progress_callback(FileProgress(0, info->size)); } return {key, info}; @@ -488,7 +488,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext S3Settings::RequestSettings request_settings; ASTPtr query; Block virtual_header; - std::function progress_callback; + std::function file_progress_callback; }; StorageS3Source::KeysIterator::KeysIterator( @@ -501,10 +501,10 @@ StorageS3Source::KeysIterator::KeysIterator( const Block & virtual_header, ContextPtr context, KeysWithInfo * read_keys, - std::function progress_callback_) + std::function file_progress_callback_) : pimpl(std::make_shared( client_, version_id_, keys_, bucket_, request_settings_, - query, virtual_header, context, read_keys, progress_callback_)) + query, virtual_header, context, read_keys, file_progress_callback_)) { } @@ -596,7 +596,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -620,13 +620,10 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k return createAsyncS3ReadBuffer(key, read_settings, object_size); } - auto buf = std::make_unique( + return std::make_unique( client, bucket, key, version_id, request_settings, read_settings, /*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0, /*restricted_seek_*/ false, object_size); - - buf->setProgressCallback(getContext()); - return buf; } std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( @@ -637,7 +634,7 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( [this, read_settings, object_size] (const std::string & path, size_t read_until_position) -> std::unique_ptr { - auto buf = std::make_unique( + return std::make_unique( client, bucket, path, @@ -649,8 +646,6 @@ std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( read_until_position, /* restricted_seek */true, object_size); - buf->setProgressCallback(getContext()); - return buf; }; auto s3_impl = std::make_unique( @@ -700,7 +695,8 @@ Chunk StorageS3Source::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - progress(num_rows, 0); + size_t chunk_size = reader.getInputFormat()->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); const auto & file_path = reader.getPath(); @@ -964,7 +960,7 @@ std::shared_ptr StorageS3::createFileIterator( ASTPtr query, const Block & virtual_block, KeysWithInfo * read_keys, - std::function progress_callback) + std::function file_progress_callback) { if (distributed_processing) { @@ -975,14 +971,14 @@ std::shared_ptr StorageS3::createFileIterator( /// Iterate through disclosed globs and make a source for each file return std::make_shared( *configuration.client, configuration.url, query, virtual_block, - local_context, read_keys, configuration.request_settings, progress_callback); + local_context, read_keys, configuration.request_settings, file_progress_callback); } else { return std::make_shared( *configuration.client, configuration.url.version_id, configuration.keys, configuration.url.bucket, configuration.request_settings, query, - virtual_block, local_context, read_keys, progress_callback); + virtual_block, local_context, read_keys, file_progress_callback); } } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 16d075a67d2d..1d589b6ef701 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -159,10 +159,12 @@ class StorageS3Source : public ISource, WithContext ReaderHolder( String path_, std::unique_ptr read_buf_, + std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) : path(std::move(path_)) , read_buf(std::move(read_buf_)) + , input_format(std::move(input_format_)) , pipeline(std::move(pipeline_)) , reader(std::move(reader_)) { @@ -183,6 +185,7 @@ class StorageS3Source : public ISource, WithContext /// reader uses pipeline, pipeline uses read_buf. reader = std::move(other.reader); pipeline = std::move(other.pipeline); + input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); path = std::move(other.path); return *this; @@ -193,9 +196,12 @@ class StorageS3Source : public ISource, WithContext const PullingPipelineExecutor * operator->() const { return reader.get(); } const String & getPath() const { return path; } + const IInputFormat * getInputFormat() const { return input_format.get(); } + private: String path; std::unique_ptr read_buf; + std::shared_ptr input_format; std::unique_ptr pipeline; std::unique_ptr reader; }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 5a8f94d07fda..119a9ef4ccd0 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -260,8 +260,7 @@ StorageURLSource::StorageURLSource( credentials, headers, glob_url, - current_uri_options.size() == 1, - context->getFileProgressCallback()); + current_uri_options.size() == 1); /// If file is empty and engine_url_skip_empty_files=1, skip it and go to the next file. } @@ -270,11 +269,11 @@ StorageURLSource::StorageURLSource( curr_uri = uri_and_buf.first; read_buf = std::move(uri_and_buf.second); - if (auto progress_callback = context->getFileProgressCallback()) + if (auto file_progress_callback = context->getFileProgressCallback()) { size_t file_size = tryGetFileSizeFromReadBuffer(*read_buf).value_or(0); LOG_DEBUG(&Poco::Logger::get("URL"), "Send file size {}", file_size); - progress_callback(FileProgress(0, file_size)); + file_progress_callback(FileProgress(0, file_size)); } // TODO: Pass max_parsing_threads and max_download_threads adjusted for num_streams. @@ -320,7 +319,8 @@ Chunk StorageURLSource::generate() if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - progress(num_rows, 0); + size_t chunk_size = input_format->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); const String & path{curr_uri.getPath()}; @@ -343,6 +343,8 @@ Chunk StorageURLSource::generate() pipeline->reset(); reader.reset(); + input_format.reset(); + read_buf.reset(); } return {}; } @@ -358,8 +360,7 @@ std::pair> StorageURLSource: Poco::Net::HTTPBasicCredentials & credentials, const HTTPHeaderEntries & headers, bool glob_url, - bool delay_initialization, - std::function file_progress_callback) + bool delay_initialization) { String first_exception_message; ReadSettings read_settings = context->getReadSettings(); @@ -401,7 +402,6 @@ std::pair> StorageURLSource: continue; } - res->setProgressCallback(file_progress_callback); return std::make_tuple(request_uri, std::move(res)); } catch (...) diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 315a5f9897bc..b1d2d0b8e353 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -195,8 +195,7 @@ class StorageURLSource : public ISource Poco::Net::HTTPBasicCredentials & credentials, const HTTPHeaderEntries & headers, bool glob_url, - bool delay_initialization, - std::function file_progress_callback = {}); + bool delay_initialization); private: using InitializeFunc = std::function; From 938a9ee3c15e0ed507470658542732ead5a93107 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Jun 2023 14:03:59 +0000 Subject: [PATCH 050/226] Better --- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 -- src/IO/ReadBufferFromS3.cpp | 2 +- src/IO/ReadWriteBufferFromHTTP.cpp | 1 + src/Storages/HDFS/HDFSCommon.h | 2 ++ src/Storages/HDFS/StorageHDFS.cpp | 10 ++++++++-- 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 46e8d914cba8..9b1c132cc010 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -99,8 +99,6 @@ size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t o n = std::min(n, mapped.getLength() - offset); memcpy(to, mapped.getData() + offset, n); - if (profile_callback) - profile_callback(ProfileInfo{.bytes_read = n}); return n; } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 2f29a7650832..d1cb1ec9ab03 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include @@ -163,6 +162,7 @@ bool ReadBufferFromS3::nextImpl() offset += working_buffer.size(); if (read_settings.remote_throttler) read_settings.remote_throttler->add(working_buffer.size(), ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + return true; } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index f38c1bcaacc8..cf1159bfb4b1 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -634,6 +634,7 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si bool cancelled; size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); + return r; } catch (const Poco::Exception & e) diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index 4588480602ac..5eb687695f29 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -57,7 +57,9 @@ static const String CONFIG_PREFIX; ~HDFSBuilderWrapper() { hdfsFreeBuilder(hdfs_builder); } HDFSBuilderWrapper(const HDFSBuilderWrapper &) = delete; + HDFSBuilderWrapper & operator=(const HDFSBuilderWrapper &) = delete; HDFSBuilderWrapper(HDFSBuilderWrapper &&) = default; + HDFSBuilderWrapper & operator=(HDFSBuilderWrapper &&) = default; hdfsBuilder * get() { return hdfs_builder; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 15632f96bc1c..57581575a27b 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -291,6 +291,12 @@ class HDFSSource::URISIterator::Impl : WithContext explicit Impl(const std::vector & uris_, ContextPtr context_) : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) { + if (!uris.empty()) + { + auto path_and_uri = getPathFromUriAndUriWithoutPath(uris[0]); + builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); + fs = createHDFSFS(builder.get()); + } } StorageHDFS::PathWithInfo next() @@ -301,8 +307,6 @@ class HDFSSource::URISIterator::Impl : WithContext auto uri = uris[current_index]; auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - HDFSBuilderWrapper builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); auto * hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); std::optional info; if (hdfs_info) @@ -318,6 +322,8 @@ class HDFSSource::URISIterator::Impl : WithContext private: std::atomic_size_t index = 0; Strings uris; + HDFSBuilderWrapper builder; + HDFSFSPtr fs; std::function file_progress_callback; }; From 21baa18969be08c68bcdfcd6f6259ada154ba8bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Jun 2023 14:16:09 +0000 Subject: [PATCH 051/226] Fix azure --- src/Storages/StorageAzureBlob.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 760c84168277..fc97dfd30160 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -804,15 +804,15 @@ StorageAzureBlobSource::Iterator::Iterator( , outer_blobs(outer_blobs_) , file_progress_callback(file_progress_callback_) { - if (keys.has_value() && blob_path_with_globs.has_value()) + if (keys_.has_value() && blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simultaneously it's a bug"); - if (!keys.has_value() && !blob_path_with_globs.has_value()) + if (!keys_.has_value() && !blob_path_with_globs.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Both keys and glob mask are not specified"); if (keys_) { - Strings all_keys = *keys; + Strings all_keys = *keys_; /// Create a virtual block with one row to construct filter if (query && virtual_header && !all_keys.empty()) From 6e0723dc828c6e6d5776e5b1761ff9b0613a3317 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Jun 2023 14:22:38 +0000 Subject: [PATCH 052/226] Fix test --- tests/queries/0_stateless/02473_infile_progress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py index 842acf2b697b..9941736107fc 100755 --- a/tests/queries/0_stateless/02473_infile_progress.py +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -32,7 +32,7 @@ ) client1.expect(prompt) client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") - client1.expect("Progress: 5.00 rows, 30.00 B.*\)") + client1.expect("Progress: 5.00 rows, 10.00 B.*\)") client1.expect(prompt) # send Ctrl-C From 207cd699579cb2da9589e30a050b0684777f415d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 02:37:10 +0200 Subject: [PATCH 053/226] Remove wrong code --- src/Functions/FunctionsComparison.h | 8 +--- src/Functions/GregorianDate.h | 47 +++++++++++++------ src/Functions/fromModifiedJulianDay.cpp | 15 +----- src/Functions/getTypeSerializationStreams.cpp | 10 +--- src/Functions/tupleHammingDistance.cpp | 2 +- src/Functions/vectorFunctions.cpp | 18 +++---- 6 files changed, 46 insertions(+), 54 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 66269f72866f..3359aca73bd1 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1178,15 +1178,9 @@ class FunctionComparison : public IFunction || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) || (arguments[0]->equals(*arguments[1])))) { - try - { - getLeastSupertype(arguments); - } - catch (const Exception &) - { + if (!tryGetLeastSupertype(arguments)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments ({}, {})" " of function {}", arguments[0]->getName(), arguments[1]->getName(), getName()); - } } if (left_tuple && right_tuple) diff --git a/src/Functions/GregorianDate.h b/src/Functions/GregorianDate.h index 63bc443fa31a..31b3c8df0dec 100644 --- a/src/Functions/GregorianDate.h +++ b/src/Functions/GregorianDate.h @@ -49,7 +49,15 @@ namespace DB /** Write the date in text form 'YYYY-MM-DD' to a buffer. */ - void write(WriteBuffer & buf) const; + void write(WriteBuffer & buf) const + { + writeImpl(buf); + } + + bool tryWrite(WriteBuffer & buf) const + { + return writeImpl(buf); + } /** Convert to a string in text form 'YYYY-MM-DD'. */ @@ -65,15 +73,18 @@ namespace DB return month_; } - uint8_t day_of_month() const noexcept /// NOLINT + uint8_t dayOfMonth() const noexcept { return day_of_month_; } private: - YearT year_; /// NOLINT - uint8_t month_; /// NOLINT - uint8_t day_of_month_; /// NOLINT + YearT year_ = 0; + uint8_t month_ = 0; + uint8_t day_of_month_ = 0; + + template + ReturnType writeImpl(WriteBuffer & buf) const; }; /** ISO 8601 Ordinal Date. YearT is an integral type which should @@ -110,8 +121,8 @@ namespace DB } private: - YearT year_; /// NOLINT - uint16_t day_of_year_; /// NOLINT + YearT year_ = 0; + uint16_t day_of_year_ = 0; }; class MonthDay @@ -135,18 +146,17 @@ namespace DB return month_; } - uint8_t day_of_month() const noexcept /// NOLINT + uint8_t dayOfMonth() const noexcept { return day_of_month_; } private: - uint8_t month_; /// NOLINT - uint8_t day_of_month_; /// NOLINT + uint8_t month_ = 0; + uint8_t day_of_month_ = 0; }; } -/* Implementation */ namespace gd { @@ -258,9 +268,10 @@ namespace DB { const OrdinalDate ord(modified_julian_day); const MonthDay md(gd::is_leap_year(ord.year()), ord.dayOfYear()); + year_ = ord.year(); month_ = md.month(); - day_of_month_ = md.day_of_month(); + day_of_month_ = md.dayOfMonth(); } template @@ -274,12 +285,16 @@ namespace DB } template - void GregorianDate::write(WriteBuffer & buf) const + template + ReturnType GregorianDate::writeImpl(WriteBuffer & buf) const { if (year_ < 0 || year_ > 9999) { - throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME, - "Impossible to stringify: year too big or small: {}", DB::toString(year_)); + if constexpr (std::is_same_v) + throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME, + "Impossible to stringify: year too big or small: {}", DB::toString(year_)); + else + return false; } else { @@ -301,6 +316,8 @@ namespace DB writeChar('0' + d / 10, buf); d %= 10; writeChar('0' + d , buf); } + + return ReturnType(); } template diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 8e76bb27ff1e..a7c2c04bf012 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -56,19 +56,8 @@ namespace DB { if constexpr (nullOnErrors) { - try - { - const GregorianDate<> gd(vec_from[i]); - gd.write(write_buffer); - (*vec_null_map_to)[i] = false; - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::CANNOT_FORMAT_DATETIME) - (*vec_null_map_to)[i] = true; - else - throw; - } + const GregorianDate<> gd(vec_from[i]); + (*vec_null_map_to)[i] = gd.tryWrite(write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } diff --git a/src/Functions/getTypeSerializationStreams.cpp b/src/Functions/getTypeSerializationStreams.cpp index 2b13f0f140d8..da9fce70ee9a 100644 --- a/src/Functions/getTypeSerializationStreams.cpp +++ b/src/Functions/getTypeSerializationStreams.cpp @@ -65,15 +65,7 @@ class FunctionGetTypeSerializationStreams : public IFunction if (!arg_string) return argument.type; - try - { - DataTypePtr type = DataTypeFactory::instance().get(arg_string->getDataAt(0).toString()); - return type; - } - catch (const DB::Exception &) - { - return argument.type; - } + return DataTypeFactory::instance().get(arg_string->getDataAt(0).toString()); } }; diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index adc063bfa816..6a78928c7daf 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -86,7 +86,7 @@ class FunctionTupleHammingDistance : public ITupleFunction auto plus_elem = plus->build({left_type, right_type}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index db907af972d6..d53d39e2f3be 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -95,7 +95,7 @@ class FunctionTupleOperator : public ITupleFunction auto elem_func = func->build(ColumnsWithTypeAndName{left, right}); types[i] = elem_func->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -181,7 +181,7 @@ class FunctionTupleNegate : public ITupleFunction auto elem_negate = negate->build(ColumnsWithTypeAndName{cur}); types[i] = elem_negate->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -258,7 +258,7 @@ class FunctionTupleOperatorByNumber : public ITupleFunction auto elem_func = func->build(ColumnsWithTypeAndName{cur, p_column}); types[i] = elem_func->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -363,7 +363,7 @@ class FunctionDotProduct : public ITupleFunction auto plus_elem = plus->build({left_type, right_type}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -467,7 +467,7 @@ class FunctionDateOrDateTimeOperationTupleOfIntervals : public ITupleFunction auto plus_elem = plus->build({left, right}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -740,7 +740,7 @@ class FunctionLNorm : public ITupleFunction auto plus_elem = plus->build({left_type, right_type}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -842,7 +842,7 @@ class FunctionLNorm : public ITupleFunction auto plus_elem = plus->build({left_type, right_type}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -993,7 +993,7 @@ class FunctionLNorm : public ITupleFunction auto max_elem = max->build({left_type, right_type}); res_type = max_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; @@ -1103,7 +1103,7 @@ class FunctionLNorm : public ITupleFunction auto plus_elem = plus->build({left_type, right_type}); res_type = plus_elem->getResultType(); } - catch (DB::Exception & e) + catch (Exception & e) { e.addMessage("While executing function {} for tuple element {}", getName(), i); throw; From bd7913a2275feb9a2e7e4a01889b1ff7297d6c24 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 02:39:25 +0200 Subject: [PATCH 054/226] Delete a line --- src/Functions/tupleHammingDistance.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index 6a78928c7daf..ffdf8c93f159 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include From bd0ce5fc0bcaaff9586e9e4549210882cc26aa64 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Jun 2023 16:34:04 +0800 Subject: [PATCH 055/226] wip --- src/Functions/substringIndex.cpp | 245 +++++++++++++++++++++++++++++++ 1 file changed, 245 insertions(+) create mode 100644 src/Functions/substringIndex.cpp diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp new file mode 100644 index 000000000000..67699304320e --- /dev/null +++ b/src/Functions/substringIndex.cpp @@ -0,0 +1,245 @@ +#include +#include +#include +#include +#include +#include +#include "base/find_symbols.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ZERO_ARRAY_OR_TUPLE_INDEX; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +template +class FunctionSubstringIndex : public IFunction +{ +public: + static constexpr auto name = is_utf8 ? "substringIndexUTF8" : "substringIndex"; + + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 3; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}", + arguments[0]->getName(), + getName()); + + if (!isString(arguments[1])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}", + arguments[1]->getName(), + getName()); + + if (!isNativeNumber(arguments[2])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}", + arguments[2]->getName(), getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_delim = arguments[1].column; + ColumnPtr column_index = arguments[2].column; + + const ColumnConst * column_delim_const = checkAndGetColumnConst(column_delim.get()); + if (!column_delim_const) + throw Exception(ErrorCodes::ILLEGAL_COLUMN , "Second argument to {} must be a constant String", getName()); + + String delim = column_delim_const->getValue(); + if constexpr (!is_utf8) + { + if (delim.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single character", getName()); + } + else + { + // TODO + } + + auto column_res = ColumnString::create(); + ColumnString::Chars & vec_res = column_res->getChars(); + ColumnString::Offsets & offsets_res = column_res->getOffsets(); + + const ColumnConst * column_string_const = checkAndGetColumnConst(column_string.get()); + if (column_string_const) + { + String str = column_string_const->getValue(); + constantVector(str, delim[0], column_index.get(), vec_res, offsets_res); + } + else + { + const auto * col_str = checkAndGetColumn(column_string.get()); + if (!col_str) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument to {} must be a String", getName()); + + bool is_index_const = isColumnConst(*column_index); + if (is_index_const) + { + Int64 index = column_index->getInt(0); + vectorConstant(col_str->getChars(), col_str->getOffsets(), delim[0], index, vec_res, offsets_res); + } + else + vectorVector(col_str->getChars(), col_str->getOffsets(), delim[0], column_index.get(), vec_res, offsets_res); + } + } + +protected: + static void vectorVector( + const ColumnString::Chars & str_data, + const ColumnString::Offsets & str_offsets, + char delim, + const IColumn * index_column, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + size_t rows = str_offsets.size(); + res_data.reserve(str_data.size() / 2); + res_offsets.reserve(rows); + + for (size_t i=0; igetInt(i); + StringRef res_ref = substringIndex(str_ref, index); + appendToResultColumn(res_ref, res_data, res_offsets); + } + } + + static void vectorConstant( + const ColumnString::Chars & str_data, + const ColumnString::Offsets & str_offsets, + char delim, + Int64 index, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + size_t rows = str_offsets.size(); + res_data.reserve(str_data.size() / 2); + res_offsets.reserve(rows); + + for (size_t i = 0; i(str_ref, index); + appendToResultColumn(res_ref, res_data, res_offsets); + } + } + + static void constantVector( + const String & str, + char delim, + const IColumn * index_column, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + size_t rows = index_column->size(); + res_data.reserve(str.size() * rows / 2); + res_offsets.reserve(rows); + + StringRef str_ref{str.data(), str.size()}; + for (size_t i=0; igetInt(i); + StringRef res_ref = substringIndex(str_ref, index); + appendToResultColumn(res_ref, res_data, res_offsets); + } + } + + static void appendToResultColumn( + const StringRef & res_ref, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + { + size_t res_offset = res_data.size(); + res_data.resize(res_offset + res_ref.size + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); + res_offset += res_ref.size; + res_data[res_offset] = 0; + ++res_offset; + + res_offsets.emplace_back(res_offset); + } + + template + static StringRef substringIndex( + const StringRef & str, + Int64 index) + { + if (index == 0) + return {str.data, 0}; + + if (index > 0) + { + const auto * end = str.data + str.size; + const auto * pos = str.data; + Int64 i = 0; + while (i < index) + { + pos = find_first_symbols(pos, end); + + if (pos != end) + { + ++pos; + ++i; + } + else + return str; + } + return {str.data, static_cast(pos - str.data)}; + } + else + { + const auto * begin = str.data; + const auto * pos = str.data + str.size; + Int64 i = 0; + while (i < index) + { + const auto * next_pos = detail::find_last_symbols_sse2(begin, pos); + + if (next_pos != pos) + { + pos = next_pos; + ++i; + } + else + return str; + } + + return {pos + 1, static_cast(str.data + str.size - pos - 1)}; + } + } +}; +} + +} + From 9ee0476d32262653d67e406a0946fa91c0bff451 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 11:59:01 +0300 Subject: [PATCH 056/226] Update src/Functions/GregorianDate.h Co-authored-by: Antonio Andelic --- src/Functions/GregorianDate.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/GregorianDate.h b/src/Functions/GregorianDate.h index 31b3c8df0dec..16fcb5ea0619 100644 --- a/src/Functions/GregorianDate.h +++ b/src/Functions/GregorianDate.h @@ -317,7 +317,7 @@ namespace DB writeChar('0' + d , buf); } - return ReturnType(); + return ReturnType(true); } template From 0de5fcfbee1d4add8c0a350392163e5f46f23f97 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Jun 2023 18:13:25 +0800 Subject: [PATCH 057/226] finish dev --- src/Functions/substringIndex.cpp | 127 +++++++++++++++++++++++++------ 1 file changed, 102 insertions(+), 25 deletions(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 67699304320e..0a5dfd006569 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -1,10 +1,14 @@ #include +#include #include +#include #include #include +#include #include -#include -#include "base/find_symbols.h" +#include +#include +#include namespace DB { @@ -67,7 +71,7 @@ class FunctionSubstringIndex : public IFunction return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { ColumnPtr column_string = arguments[0].column; ColumnPtr column_delim = arguments[1].column; @@ -85,7 +89,8 @@ class FunctionSubstringIndex : public IFunction } else { - // TODO + if (UTF8::countCodePoints(reinterpret_cast(delim.data()), delim.size()) != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single UTF-8 character", getName()); } auto column_res = ColumnString::create(); @@ -96,7 +101,7 @@ class FunctionSubstringIndex : public IFunction if (column_string_const) { String str = column_string_const->getValue(); - constantVector(str, delim[0], column_index.get(), vec_res, offsets_res); + constantVector(str, delim, column_index.get(), vec_res, offsets_res); } else { @@ -108,10 +113,10 @@ class FunctionSubstringIndex : public IFunction if (is_index_const) { Int64 index = column_index->getInt(0); - vectorConstant(col_str->getChars(), col_str->getOffsets(), delim[0], index, vec_res, offsets_res); + vectorConstant(col_str->getChars(), col_str->getOffsets(), delim, index, vec_res, offsets_res); } else - vectorVector(col_str->getChars(), col_str->getOffsets(), delim[0], column_index.get(), vec_res, offsets_res); + vectorVector(col_str->getChars(), col_str->getOffsets(), delim, column_index.get(), vec_res, offsets_res); } } @@ -119,7 +124,7 @@ class FunctionSubstringIndex : public IFunction static void vectorVector( const ColumnString::Chars & str_data, const ColumnString::Offsets & str_offsets, - char delim, + const String & delim, const IColumn * index_column, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) @@ -128,11 +133,15 @@ class FunctionSubstringIndex : public IFunction res_data.reserve(str_data.size() / 2); res_offsets.reserve(rows); - for (size_t i=0; i searcher + = !is_utf8 ? nullptr : std::make_unique(delim); + + for (size_t i = 0; i < rows; ++i) { StringRef str_ref{&str_data[str_offsets[i]], str_offsets[i] - str_offsets[i - 1] - 1}; Int64 index = index_column->getInt(i); - StringRef res_ref = substringIndex(str_ref, index); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -140,7 +149,7 @@ class FunctionSubstringIndex : public IFunction static void vectorConstant( const ColumnString::Chars & str_data, const ColumnString::Offsets & str_offsets, - char delim, + const String & delim, Int64 index, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) @@ -149,17 +158,21 @@ class FunctionSubstringIndex : public IFunction res_data.reserve(str_data.size() / 2); res_offsets.reserve(rows); + std::unique_ptr searcher + = !is_utf8 ? nullptr : std::make_unique(delim); + for (size_t i = 0; i(str_ref, index); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); appendToResultColumn(res_ref, res_data, res_offsets); } } static void constantVector( const String & str, - char delim, + const String & delim, const IColumn * index_column, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) @@ -168,11 +181,15 @@ class FunctionSubstringIndex : public IFunction res_data.reserve(str.size() * rows / 2); res_offsets.reserve(rows); + std::unique_ptr searcher + = !is_utf8 ? nullptr : std::make_unique(delim); + StringRef str_ref{str.data(), str.size()}; for (size_t i=0; igetInt(i); - StringRef res_ref = substringIndex(str_ref, index); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -190,18 +207,68 @@ class FunctionSubstringIndex : public IFunction res_offsets.emplace_back(res_offset); } + static StringRef substringIndexUTF8( + const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 index) + { + if (index == 0) + return {str_ref.data, 0}; + + const auto * begin = reinterpret_cast(str_ref.data); + const auto * end = reinterpret_cast(str_ref.data + str_ref.size); + const auto * pos = begin; + if (index > 0) + { + Int64 i = 0; + while (i < index) + { + pos = searcher->search(pos, end - pos); + + if (pos != end) + { + pos += delim.size(); + ++i; + } + else + return str_ref; + } + return {begin, static_cast(pos - begin - delim.size())}; + } + else + { + Int64 total = 0; + while (pos < end && end != (pos = searcher->search(pos, end - pos))) + { + pos += delim.size(); + ++total; + } + + if (total + index < 0) + return str_ref; + + Int64 index_from_left = total + 1 + index; + pos = begin; + Int64 i = 0; + while (pos < end && end != (pos = searcher->search(pos, end - pos)) && i < index_from_left) + { + pos += delim.size(); + ++i; + } + return {pos, static_cast(end - pos)}; + } + } + template static StringRef substringIndex( - const StringRef & str, + const StringRef & str_ref, Int64 index) { if (index == 0) - return {str.data, 0}; + return {str_ref.data, 0}; if (index > 0) { - const auto * end = str.data + str.size; - const auto * pos = str.data; + const auto * end = str_ref.data + str_ref.size; + const auto * pos = str_ref.data; Int64 i = 0; while (i < index) { @@ -213,18 +280,18 @@ class FunctionSubstringIndex : public IFunction ++i; } else - return str; + return str_ref; } - return {str.data, static_cast(pos - str.data)}; + return {str_ref.data, static_cast(pos - str_ref.data - 1)}; } else { - const auto * begin = str.data; - const auto * pos = str.data + str.size; + const auto * begin = str_ref.data; + const auto * pos = str_ref.data + str_ref.size; Int64 i = 0; while (i < index) { - const auto * next_pos = detail::find_last_symbols_sse2(begin, pos); + const auto * next_pos = ::detail::find_last_symbols_sse2(begin, pos); if (next_pos != pos) { @@ -232,14 +299,24 @@ class FunctionSubstringIndex : public IFunction ++i; } else - return str; + return str_ref; } - return {pos + 1, static_cast(str.data + str.size - pos - 1)}; + return {pos + 1, static_cast(str_ref.data + str_ref.size - pos - 1)}; } } }; } + +REGISTER_FUNCTION(SubstringIndex) +{ + factory.registerFunction>(); /// substringIndex + factory.registerFunction>(); /// substringIndexUTF8 + + factory.registerAlias("SUBSTRING_INDEX", "substringIndex", FunctionFactory::CaseInsensitive); +} + + } From ae7a586aea59deb84a7355021b06eb3b35d876f7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Jun 2023 10:45:52 +0800 Subject: [PATCH 058/226] fix bugs and add uts --- src/Functions/substringIndex.cpp | 459 +++++++++--------- .../02798_substring_index.reference | 155 ++++++ .../0_stateless/02798_substring_index.sql | 93 ++++ 3 files changed, 477 insertions(+), 230 deletions(-) create mode 100644 tests/queries/0_stateless/02798_substring_index.reference create mode 100644 tests/queries/0_stateless/02798_substring_index.sql diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 0a5dfd006569..1fca3bbed146 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -25,287 +25,287 @@ namespace ErrorCodes namespace { -template -class FunctionSubstringIndex : public IFunction -{ -public: - static constexpr auto name = is_utf8 ? "substringIndexUTF8" : "substringIndex"; - - - static FunctionPtr create(ContextPtr) + template + class FunctionSubstringIndex : public IFunction { - return std::make_shared(); - } + public: + static constexpr auto name = is_utf8 ? "substringIndexUTF8" : "substringIndex"; - String getName() const override - { - return name; - } - size_t getNumberOfArguments() const override { return 3; } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + String getName() const override { return name; } - bool useDefaultImplementationForConstants() const override { return true; } + size_t getNumberOfArguments() const override { return 3; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {}", - arguments[0]->getName(), - getName()); - - if (!isString(arguments[1])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}", - arguments[1]->getName(), - getName()); - - if (!isNativeNumber(arguments[2])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}", - arguments[2]->getName(), getName()); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - ColumnPtr column_string = arguments[0].column; - ColumnPtr column_delim = arguments[1].column; - ColumnPtr column_index = arguments[2].column; + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - const ColumnConst * column_delim_const = checkAndGetColumnConst(column_delim.get()); - if (!column_delim_const) - throw Exception(ErrorCodes::ILLEGAL_COLUMN , "Second argument to {} must be a constant String", getName()); + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - String delim = column_delim_const->getValue(); - if constexpr (!is_utf8) + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (delim.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single character", getName()); + if (!isString(arguments[0])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}", + arguments[0]->getName(), + getName()); + + if (!isString(arguments[1])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}", + arguments[1]->getName(), + getName()); + + if (!isNativeNumber(arguments[2])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of third argument of function {}", + arguments[2]->getName(), + getName()); + + return std::make_shared(); } - else + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - if (UTF8::countCodePoints(reinterpret_cast(delim.data()), delim.size()) != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single UTF-8 character", getName()); - } + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_delim = arguments[1].column; + ColumnPtr column_index = arguments[2].column; - auto column_res = ColumnString::create(); - ColumnString::Chars & vec_res = column_res->getChars(); - ColumnString::Offsets & offsets_res = column_res->getOffsets(); + const ColumnConst * column_delim_const = checkAndGetColumnConst(column_delim.get()); + if (!column_delim_const) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument to {} must be a constant String", getName()); - const ColumnConst * column_string_const = checkAndGetColumnConst(column_string.get()); - if (column_string_const) - { - String str = column_string_const->getValue(); - constantVector(str, delim, column_index.get(), vec_res, offsets_res); - } - else - { - const auto * col_str = checkAndGetColumn(column_string.get()); - if (!col_str) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument to {} must be a String", getName()); + String delim = column_delim_const->getValue(); + if constexpr (!is_utf8) + { + if (delim.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single character", getName()); + } + else + { + if (UTF8::countCodePoints(reinterpret_cast(delim.data()), delim.size()) != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument to {} must be a single UTF-8 character", getName()); + } + + auto column_res = ColumnString::create(); + ColumnString::Chars & vec_res = column_res->getChars(); + ColumnString::Offsets & offsets_res = column_res->getOffsets(); - bool is_index_const = isColumnConst(*column_index); - if (is_index_const) + const ColumnConst * column_string_const = checkAndGetColumnConst(column_string.get()); + if (column_string_const) { - Int64 index = column_index->getInt(0); - vectorConstant(col_str->getChars(), col_str->getOffsets(), delim, index, vec_res, offsets_res); + String str = column_string_const->getValue(); + constantVector(str, delim, column_index.get(), vec_res, offsets_res); } else - vectorVector(col_str->getChars(), col_str->getOffsets(), delim, column_index.get(), vec_res, offsets_res); - } - } - -protected: - static void vectorVector( - const ColumnString::Chars & str_data, - const ColumnString::Offsets & str_offsets, - const String & delim, - const IColumn * index_column, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) - { - size_t rows = str_offsets.size(); - res_data.reserve(str_data.size() / 2); - res_offsets.reserve(rows); + { + const auto * col_str = checkAndGetColumn(column_string.get()); + if (!col_str) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument to {} must be a String", getName()); - std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim); + bool is_index_const = isColumnConst(*column_index); + if (is_index_const) + { + Int64 index = column_index->getInt(0); + vectorConstant(col_str, delim, index, vec_res, offsets_res); + } + else + vectorVector(col_str, delim, column_index.get(), vec_res, offsets_res); + } + return column_res; + } - for (size_t i = 0; i < rows; ++i) + protected: + static void vectorVector( + const ColumnString * str_column, + const String & delim, + const IColumn * index_column, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) { - StringRef str_ref{&str_data[str_offsets[i]], str_offsets[i] - str_offsets[i - 1] - 1}; - Int64 index = index_column->getInt(i); - StringRef res_ref - = !is_utf8 ? substringIndex(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); - appendToResultColumn(res_ref, res_data, res_offsets); - } - } - - static void vectorConstant( - const ColumnString::Chars & str_data, - const ColumnString::Offsets & str_offsets, - const String & delim, - Int64 index, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) - { - size_t rows = str_offsets.size(); - res_data.reserve(str_data.size() / 2); - res_offsets.reserve(rows); + size_t rows = str_column->size(); + res_data.reserve(str_column->getChars().size() / 2); + res_offsets.reserve(rows); - std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim); + std::unique_ptr searcher + = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); - for (size_t i = 0; i(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); - appendToResultColumn(res_ref, res_data, res_offsets); + for (size_t i = 0; i < rows; ++i) + { + StringRef str_ref = str_column->getDataAt(i); + Int64 index = index_column->getInt(i); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + appendToResultColumn(res_ref, res_data, res_offsets); + } } - } - - static void constantVector( - const String & str, - const String & delim, - const IColumn * index_column, - ColumnString::Chars & res_data, - ColumnString::Offsets & res_offsets) - { - size_t rows = index_column->size(); - res_data.reserve(str.size() * rows / 2); - res_offsets.reserve(rows); - - std::unique_ptr searcher - = !is_utf8 ? nullptr : std::make_unique(delim); - StringRef str_ref{str.data(), str.size()}; - for (size_t i=0; igetInt(i); - StringRef res_ref - = !is_utf8 ? substringIndex(str_ref, index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); - appendToResultColumn(res_ref, res_data, res_offsets); - } - } + size_t rows = str_column->size(); + res_data.reserve(str_column->getChars().size() / 2); + res_offsets.reserve(rows); - static void appendToResultColumn( - const StringRef & res_ref, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) - { - size_t res_offset = res_data.size(); - res_data.resize(res_offset + res_ref.size + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); - res_offset += res_ref.size; - res_data[res_offset] = 0; - ++res_offset; - - res_offsets.emplace_back(res_offset); - } - - static StringRef substringIndexUTF8( - const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 index) - { - if (index == 0) - return {str_ref.data, 0}; + std::unique_ptr searcher + = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); - const auto * begin = reinterpret_cast(str_ref.data); - const auto * end = reinterpret_cast(str_ref.data + str_ref.size); - const auto * pos = begin; - if (index > 0) - { - Int64 i = 0; - while (i < index) + for (size_t i = 0; i < rows; ++i) { - pos = searcher->search(pos, end - pos); - - if (pos != end) - { - pos += delim.size(); - ++i; - } - else - return str_ref; + StringRef str_ref = str_column->getDataAt(i); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + std::cout << "result:" << res_ref.toString() << std::endl; + appendToResultColumn(res_ref, res_data, res_offsets); } - return {begin, static_cast(pos - begin - delim.size())}; } - else + + static void constantVector( + const String & str, + const String & delim, + const IColumn * index_column, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) { - Int64 total = 0; - while (pos < end && end != (pos = searcher->search(pos, end - pos))) - { - pos += delim.size(); - ++total; - } + size_t rows = index_column->size(); + res_data.reserve(str.size() * rows / 2); + res_offsets.reserve(rows); - if (total + index < 0) - return str_ref; + std::unique_ptr searcher + = !is_utf8 ? nullptr : std::make_unique(delim.data(), delim.size()); - Int64 index_from_left = total + 1 + index; - pos = begin; - Int64 i = 0; - while (pos < end && end != (pos = searcher->search(pos, end - pos)) && i < index_from_left) + StringRef str_ref{str.data(), str.size()}; + for (size_t i = 0; i < rows; ++i) { - pos += delim.size(); - ++i; + Int64 index = index_column->getInt(i); + StringRef res_ref + = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + appendToResultColumn(res_ref, res_data, res_offsets); } - return {pos, static_cast(end - pos)}; } - } - template - static StringRef substringIndex( - const StringRef & str_ref, - Int64 index) - { - if (index == 0) - return {str_ref.data, 0}; + static void appendToResultColumn(const StringRef & res_ref, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + { + size_t res_offset = res_data.size(); + res_data.resize(res_offset + res_ref.size + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); + res_offset += res_ref.size; + res_data[res_offset] = 0; + ++res_offset; + + res_offsets.emplace_back(res_offset); + } - if (index > 0) + static StringRef substringIndexUTF8( + const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 index) { - const auto * end = str_ref.data + str_ref.size; - const auto * pos = str_ref.data; - Int64 i = 0; - while (i < index) + std::cout << "str:" << str_ref.toString() << ", delim" << delim << ",index:" << index << std::endl; + + if (index == 0) + return {str_ref.data, 0}; + + const auto * begin = reinterpret_cast(str_ref.data); + const auto * end = reinterpret_cast(str_ref.data + str_ref.size); + const auto * pos = begin; + if (index > 0) + { + Int64 i = 0; + while (i < index) + { + pos = searcher->search(pos, end - pos); + + if (pos != end) + { + pos += delim.size(); + ++i; + } + else + return str_ref; + } + return {begin, static_cast(pos - begin - delim.size())}; + } + else { - pos = find_first_symbols(pos, end); + Int64 total = 0; + while (pos < end && end != (pos = searcher->search(pos, end - pos))) + { + pos += delim.size(); + ++total; + } - if (pos != end) + if (total + index < 0) + return str_ref; + + Int64 index_from_left = total + 1 + index; + std::cout << "total:" << total << ", index_from_left" << index_from_left << std::endl; + pos = begin; + Int64 i = 0; + while (i < index_from_left && pos < end && end != (pos = searcher->search(pos, end - pos))) { - ++pos; + pos += delim.size(); ++i; + std::cout << "pos offset:" << pos - begin << ", total size:" << end - begin << std::endl; } - else - return str_ref; + std::cout << "pos offset:" << pos - begin << ", size:" << end - pos << std::endl; + StringRef res = {pos, static_cast(end - pos)}; + std::cout << "result:" << res.toString() << std::endl; + return res; } - return {str_ref.data, static_cast(pos - str_ref.data - 1)}; } - else + + static StringRef substringIndex(const StringRef & str_ref, char delim, Int64 index) { - const auto * begin = str_ref.data; - const auto * pos = str_ref.data + str_ref.size; - Int64 i = 0; - while (i < index) - { - const auto * next_pos = ::detail::find_last_symbols_sse2(begin, pos); + std::cout << "str:" << str_ref.toString() << ", delim" << delim << ",index:" << index << std::endl; + + if (index == 0) + return {str_ref.data, 0}; - if (next_pos != pos) + if (index > 0) + { + const auto * end = str_ref.data + str_ref.size; + const auto * pos = str_ref.data; + Int64 i = 0; + while (i < index) { - pos = next_pos; - ++i; + pos = std::find(pos, end, delim); + if (pos != end) + { + ++pos; + ++i; + } + else + return str_ref; } - else - return str_ref; + return {str_ref.data, static_cast(pos - str_ref.data - 1)}; + } + else + { + const auto * begin = str_ref.data; + const auto * pos = str_ref.data + str_ref.size; + Int64 i = 0; + while (i + index < 0) + { + --pos; + while (pos >= begin && *pos != delim) + --pos; + + if (pos >= begin) + ++i; + else + return str_ref; + } + return {pos + 1, static_cast(str_ref.data + str_ref.size - pos - 1)}; } - - return {pos + 1, static_cast(str_ref.data + str_ref.size - pos - 1)}; } - } -}; + }; } @@ -319,4 +319,3 @@ REGISTER_FUNCTION(SubstringIndex) } - diff --git a/tests/queries/0_stateless/02798_substring_index.reference b/tests/queries/0_stateless/02798_substring_index.reference new file mode 100644 index 000000000000..a3084509c121 --- /dev/null +++ b/tests/queries/0_stateless/02798_substring_index.reference @@ -0,0 +1,155 @@ +-- { echoOn } +select substringIndex('www.clickhouse.com', '.', -4); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', -3); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', -2); +clickhouse.com +select substringIndex('www.clickhouse.com', '.', -1); +com +select substringIndex('www.clickhouse.com', '.', 0); + +select substringIndex('www.clickhouse.com', '.', 1); +www +select substringIndex('www.clickhouse.com', '.', 2); +www.clickhouse +select substringIndex('www.clickhouse.com', '.', 3); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', 4); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', -4); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', -3); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', -2); +clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', -1); +com +select substringIndex(materialize('www.clickhouse.com'), '.', 0); + +select substringIndex(materialize('www.clickhouse.com'), '.', 1); +www +select substringIndex(materialize('www.clickhouse.com'), '.', 2); +www.clickhouse +select substringIndex(materialize('www.clickhouse.com'), '.', 3); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', 4); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-4)); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-3)); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-2)); +clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-1)); +com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(0)); + +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(1)); +www +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(2)); +www.clickhouse +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(3)); +www.clickhouse.com +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(4)); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', materialize(-4)); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', materialize(-3)); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', materialize(-2)); +clickhouse.com +select substringIndex('www.clickhouse.com', '.', materialize(-1)); +com +select substringIndex('www.clickhouse.com', '.', materialize(0)); + +select substringIndex('www.clickhouse.com', '.', materialize(1)); +www +select substringIndex('www.clickhouse.com', '.', materialize(2)); +www.clickhouse +select substringIndex('www.clickhouse.com', '.', materialize(3)); +www.clickhouse.com +select substringIndex('www.clickhouse.com', '.', materialize(4)); +www.clickhouse.com +select SUBSTRING_INDEX('www.clickhouse.com', '.', 2); +www.clickhouse +select substringIndex('www.clickhouse.com', '..', 2); -- { serverError BAD_ARGUMENTS } +select substringIndex('www.clickhouse.com', '', 2); -- { serverError BAD_ARGUMENTS } +select substringIndex('www.clickhouse.com', materialize('.'), 2); -- { serverError ILLEGAL_COLUMN } +select substringIndex('www.clickhouse.com', '.', cast(2 as Int128)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select substringIndexUTF8('富强,民主,文明', ',', -4); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', -3); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', -2); +民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', -1); +文明 +select substringIndexUTF8('富强,民主,文明', ',', 0); + +select substringIndexUTF8('富强,民主,文明', ',', 1); +富强 +select substringIndexUTF8('富强,民主,文明', ',', 2); +富强,民主 +select substringIndexUTF8('富强,民主,文明', ',', 3); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', 4); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -4); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -3); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -2); +民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -1); +文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 0); + +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 1); +富强 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 2); +富强,民主 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 3); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 4); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(-4)); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(-3)); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(-2)); +民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(-1)); +文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(0)); + +select substringIndexUTF8('富强,民主,文明', ',', materialize(1)); +富强 +select substringIndexUTF8('富强,民主,文明', ',', materialize(2)); +富强,民主 +select substringIndexUTF8('富强,民主,文明', ',', materialize(3)); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',', materialize(4)); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-4)); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-3)); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-2)); +民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-1)); +文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(0)); + +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(1)); +富强 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(2)); +富强,民主 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(3)); +富强,民主,文明 +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(4)); +富强,民主,文明 +select substringIndexUTF8('富强,民主,文明', ',,', 2); -- { serverError BAD_ARGUMENTS } +select substringIndexUTF8('富强,民主,文明', '', 2); -- { serverError BAD_ARGUMENTS } +select substringIndexUTF8('富强,民主,文明', materialize(','), 2); -- { serverError ILLEGAL_COLUMN } +select substringIndexUTF8('富强,民主,文明', ',', cast(2 as Int128)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02798_substring_index.sql b/tests/queries/0_stateless/02798_substring_index.sql new file mode 100644 index 000000000000..520775e89706 --- /dev/null +++ b/tests/queries/0_stateless/02798_substring_index.sql @@ -0,0 +1,93 @@ +-- { echoOn } +select substringIndex('www.clickhouse.com', '.', -4); +select substringIndex('www.clickhouse.com', '.', -3); +select substringIndex('www.clickhouse.com', '.', -2); +select substringIndex('www.clickhouse.com', '.', -1); +select substringIndex('www.clickhouse.com', '.', 0); +select substringIndex('www.clickhouse.com', '.', 1); +select substringIndex('www.clickhouse.com', '.', 2); +select substringIndex('www.clickhouse.com', '.', 3); +select substringIndex('www.clickhouse.com', '.', 4); + +select substringIndex(materialize('www.clickhouse.com'), '.', -4); +select substringIndex(materialize('www.clickhouse.com'), '.', -3); +select substringIndex(materialize('www.clickhouse.com'), '.', -2); +select substringIndex(materialize('www.clickhouse.com'), '.', -1); +select substringIndex(materialize('www.clickhouse.com'), '.', 0); +select substringIndex(materialize('www.clickhouse.com'), '.', 1); +select substringIndex(materialize('www.clickhouse.com'), '.', 2); +select substringIndex(materialize('www.clickhouse.com'), '.', 3); +select substringIndex(materialize('www.clickhouse.com'), '.', 4); + +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-4)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-3)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-2)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(-1)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(0)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(1)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(2)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(3)); +select substringIndex(materialize('www.clickhouse.com'), '.', materialize(4)); + +select substringIndex('www.clickhouse.com', '.', materialize(-4)); +select substringIndex('www.clickhouse.com', '.', materialize(-3)); +select substringIndex('www.clickhouse.com', '.', materialize(-2)); +select substringIndex('www.clickhouse.com', '.', materialize(-1)); +select substringIndex('www.clickhouse.com', '.', materialize(0)); +select substringIndex('www.clickhouse.com', '.', materialize(1)); +select substringIndex('www.clickhouse.com', '.', materialize(2)); +select substringIndex('www.clickhouse.com', '.', materialize(3)); +select substringIndex('www.clickhouse.com', '.', materialize(4)); + +select SUBSTRING_INDEX('www.clickhouse.com', '.', 2); + +select substringIndex('www.clickhouse.com', '..', 2); -- { serverError BAD_ARGUMENTS } +select substringIndex('www.clickhouse.com', '', 2); -- { serverError BAD_ARGUMENTS } +select substringIndex('www.clickhouse.com', materialize('.'), 2); -- { serverError ILLEGAL_COLUMN } +select substringIndex('www.clickhouse.com', '.', cast(2 as Int128)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +select substringIndexUTF8('富强,民主,文明', ',', -4); +select substringIndexUTF8('富强,民主,文明', ',', -3); +select substringIndexUTF8('富强,民主,文明', ',', -2); +select substringIndexUTF8('富强,民主,文明', ',', -1); +select substringIndexUTF8('富强,民主,文明', ',', 0); +select substringIndexUTF8('富强,民主,文明', ',', 1); +select substringIndexUTF8('富强,民主,文明', ',', 2); +select substringIndexUTF8('富强,民主,文明', ',', 3); +select substringIndexUTF8('富强,民主,文明', ',', 4); + +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -4); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -3); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -2); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', -1); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 0); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 1); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 2); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 3); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', 4); + +select substringIndexUTF8('富强,民主,文明', ',', materialize(-4)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(-3)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(-2)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(-1)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(0)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(1)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(2)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(3)); +select substringIndexUTF8('富强,民主,文明', ',', materialize(4)); + +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-4)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-3)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-2)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(-1)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(0)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(1)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(2)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(3)); +select substringIndexUTF8(materialize('富强,民主,文明'), ',', materialize(4)); + +select substringIndexUTF8('富强,民主,文明', ',,', 2); -- { serverError BAD_ARGUMENTS } +select substringIndexUTF8('富强,民主,文明', '', 2); -- { serverError BAD_ARGUMENTS } +select substringIndexUTF8('富强,民主,文明', materialize(','), 2); -- { serverError ILLEGAL_COLUMN } +select substringIndexUTF8('富强,民主,文明', ',', cast(2 as Int128)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +-- { echoOff } From 70e49cb31c0ff80ffc6c8e6ab5687b24af659ad1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Jun 2023 11:28:20 +0800 Subject: [PATCH 059/226] add docs --- .../functions/string-functions.md | 36 +++++++++++++++++++ src/Functions/substringIndex.cpp | 16 ++------- ...new_functions_must_be_documented.reference | 1 + 3 files changed, 39 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5175bbf0615f..5197b786884a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -573,6 +573,42 @@ Alias: Like `substring` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +## substringIndex(s, delim, index) + +Returns the substring of `s` before `index` occurrences of the delimiter `delim`, as in Spark or MySQL. + +**Syntax** + +```sql +substringIndex(s, delim, index) +``` +Alias: `SUBSTRING_INDEX` + + +**Arguments** + +- s: The string to extract substring from. [String](../../sql-reference/data-types/string.md). +- delim: The character to split. [String](../../sql-reference/data-types/string.md). +- index: The number of occurrences of the delimiter to count before extracting the substring. If index is positive, everything to the left of the final delimiter (counting from the left) is returned. If index is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) + +**Example** + +``` sql +SELECT substringIndex('www.clickhouse.com', '.', 2) +``` + +Result: +``` +┌─substringIndex('www.clickhouse.com', '.', 2)─┐ +│ www.clickhouse │ +└──────────────────────────────────────────────┘ +``` + +## substringIndexUTF8(s, delim, index) + +Like `substringIndex` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + ## appendTrailingCharIfAbsent Appends character `c` to string `s` if `s` is non-empty and does not end with character `c`. diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 1fca3bbed146..fbb20b245f6d 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -17,8 +17,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } @@ -163,7 +161,6 @@ namespace StringRef str_ref = str_column->getDataAt(i); StringRef res_ref = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); - std::cout << "result:" << res_ref.toString() << std::endl; appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -207,8 +204,6 @@ namespace static StringRef substringIndexUTF8( const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 index) { - std::cout << "str:" << str_ref.toString() << ", delim" << delim << ",index:" << index << std::endl; - if (index == 0) return {str_ref.data, 0}; @@ -244,27 +239,20 @@ namespace if (total + index < 0) return str_ref; - Int64 index_from_left = total + 1 + index; - std::cout << "total:" << total << ", index_from_left" << index_from_left << std::endl; pos = begin; Int64 i = 0; + Int64 index_from_left = total + 1 + index; while (i < index_from_left && pos < end && end != (pos = searcher->search(pos, end - pos))) { pos += delim.size(); ++i; - std::cout << "pos offset:" << pos - begin << ", total size:" << end - begin << std::endl; } - std::cout << "pos offset:" << pos - begin << ", size:" << end - pos << std::endl; - StringRef res = {pos, static_cast(end - pos)}; - std::cout << "result:" << res.toString() << std::endl; - return res; + return {pos, static_cast(end - pos)}; } } static StringRef substringIndex(const StringRef & str_ref, char delim, Int64 index) { - std::cout << "str:" << str_ref.toString() << ", delim" << delim << ",index:" << index << std::endl; - if (index == 0) return {str_ref.data, 0}; diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index b5c133988e6b..6c904d6fc055 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -666,6 +666,7 @@ startsWith subBitmap substring substringUTF8 +substringIndex subtractDays subtractHours subtractMicroseconds From 3f73d3f48aa679dc689dea6e49594752461e4d8b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Jun 2023 11:33:39 +0800 Subject: [PATCH 060/226] fix failed check --- .../02415_all_new_functions_must_be_documented.reference | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 6c904d6fc055..a2621949d0d9 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -665,8 +665,9 @@ sqrt startsWith subBitmap substring -substringUTF8 substringIndex +substringIndexUTF8 +substringUTF8 subtractDays subtractHours subtractMicroseconds From 375f7abfeba866ae7956e58e9bd1bf364b972ea5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Jun 2023 12:27:59 +0800 Subject: [PATCH 061/226] fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 00d047121e6e..9af484172504 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -210,6 +210,7 @@ Decrypted Deduplicate Deduplication DelayedInserts +delim DeliveryTag DeltaLake Denormalize @@ -834,6 +835,8 @@ Subexpression Submodules Subqueries Substrings +substringIndex +substringIndexUTF SummingMergeTree SuperSet Superset From a2c9e26b231d15a96818440c001897daa6e4004d Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 28 Jun 2023 09:31:44 +0000 Subject: [PATCH 062/226] Add peak memory usage (prototype) --- src/Client/ClientBase.cpp | 2 ++ src/Common/MemoryTracker.h | 1 + src/Common/ProgressIndication.cpp | 10 ++++++---- src/Common/ProgressIndication.h | 2 ++ src/Interpreters/ProfileEventsExt.cpp | 12 ++++++++++-- src/Interpreters/ProfileEventsExt.h | 1 + 6 files changed, 22 insertions(+), 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 34b3b1e228a0..ecc873da068a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1189,6 +1189,8 @@ void ClientBase::onProfileEvents(Block & block) thread_times[host_name].system_ms = value; else if (event_name == MemoryTracker::USAGE_EVENT_NAME) thread_times[host_name].memory_usage = value; + else if (event_name == MemoryTracker::PEAK_USAGE_EVENT_NAME) + thread_times[host_name].peak_memory_usage = value; } progress_indication.updateThreadEventData(thread_times); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4e29d40c953d..3ea1ea8702c4 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -95,6 +95,7 @@ class MemoryTracker public: static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage"; + static constexpr auto PEAK_USAGE_EVENT_NAME = "MemoryTrackerPeakUsage"; explicit MemoryTracker(VariableContext level_ = VariableContext::Thread); explicit MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 61b60060430a..d9564f0946ec 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -83,7 +83,7 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const [](MemoryUsage const & acc, auto const & host_data) { UInt64 host_usage = host_data.second.memory_usage; - return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage), .peak = std::max(acc.peak, host_data.second.peak_memory_usage)}; }); } @@ -99,8 +99,8 @@ void ProgressIndication::writeFinalProgress() if (elapsed_ns) std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., " << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; - else - std::cout << ". "; + auto peak_memory_usage = getMemoryUsage().peak; + std::cout << ".\nPeak memory usage (for query) " << formatReadableSizeWithDecimalSuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) @@ -152,7 +152,7 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) std::string profiling_msg; double cpu_usage = getCPUUsage(); - auto [memory_usage, max_host_usage] = getMemoryUsage(); + auto [memory_usage, max_host_usage, peak_usage] = getMemoryUsage(); if (cpu_usage > 0 || memory_usage > 0) { @@ -166,6 +166,8 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) if (memory_usage > 0) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM"; + // profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << ", " + // << formatReadableSizeWithDecimalSuffix(peak_usage) << " Peak RAM"; if (max_host_usage < memory_usage) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(max_host_usage) << " max/host"; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index af5d69c0255d..e3c73e8e7e58 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -22,6 +22,7 @@ struct ThreadEventData UInt64 user_ms = 0; UInt64 system_ms = 0; UInt64 memory_usage = 0; + UInt64 peak_memory_usage = 0; }; using HostToTimesMap = std::unordered_map; @@ -64,6 +65,7 @@ class ProgressIndication { UInt64 total = 0; UInt64 max = 0; + UInt64 peak = 0; }; MemoryUsage getMemoryUsage() const; diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index bf8d060bd3cb..bd421ae8e337 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -86,9 +86,16 @@ static void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::Mutabl columns[i++]->insert(static_cast(snapshot.current_time)); columns[i++]->insert(static_cast(snapshot.thread_id)); columns[i++]->insert(Type::GAUGE); - columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); - columns[i++]->insert(snapshot.memory_usage); + columns[i]->insert(snapshot.memory_usage); + + i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(static_cast(snapshot.current_time)); + columns[i++]->insert(static_cast(snapshot.thread_id)); + columns[i++]->insert(Type::GAUGE); + columns[i++]->insertData(MemoryTracker::PEAK_USAGE_EVENT_NAME, strlen(MemoryTracker::PEAK_USAGE_EVENT_NAME)); + columns[i]->insert(snapshot.peak_memory_usage); } void getProfileEvents( @@ -121,6 +128,7 @@ void getProfileEvents( group_snapshot.thread_id = 0; group_snapshot.current_time = time(nullptr); group_snapshot.memory_usage = thread_group->memory_tracker.get(); + group_snapshot.peak_memory_usage = thread_group->memory_tracker.getPeak(); auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); auto prev_group_snapshot = last_sent_snapshots.find(0); group_snapshot.counters = diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 7d9fc512d15b..cc338530510d 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -16,6 +16,7 @@ struct ProfileEventsSnapshot UInt64 thread_id; CountersIncrement counters; Int64 memory_usage; + Int64 peak_memory_usage; time_t current_time; }; From a41ec1221e8b1d33560af877ad6711be8f93de9c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 28 Jun 2023 13:50:53 +0000 Subject: [PATCH 063/226] Http prototype --- src/Common/ProgressIndication.cpp | 4 ++-- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 7 ++++++- src/Server/HTTP/WriteBufferFromHTTPServerResponse.h | 5 +++++ src/Server/HTTPHandler.cpp | 7 ++++++- 4 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index d9564f0946ec..29766fd1a148 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -100,7 +100,7 @@ void ProgressIndication::writeFinalProgress() std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., " << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; auto peak_memory_usage = getMemoryUsage().peak; - std::cout << ".\nPeak memory usage (for query) " << formatReadableSizeWithDecimalSuffix(peak_memory_usage) << "."; + std::cout << ".\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) @@ -166,7 +166,7 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) if (memory_usage > 0) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM"; - // profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << ", " + // profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << ", " // << formatReadableSizeWithDecimalSuffix(peak_usage) << " Peak RAM"; if (max_host_usage < memory_usage) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(max_host_usage) << " max/host"; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index c8015cfd1850..1e6d520de0ab 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -38,7 +38,7 @@ void WriteBufferFromHTTPServerResponse::writeHeaderSummary() accumulated_progress.writeJSON(progress_string_writer); if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << "\r\n" << std::flush; + *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << " Mem " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "\r\n" << std::flush; } void WriteBufferFromHTTPServerResponse::writeHeaderProgress() @@ -169,6 +169,11 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) } } +void WriteBufferFromHTTPServerResponse::onMemoryUsage(Int64 usage) +{ + peak_memory_usage = usage; +} + WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index ce677616755d..f849a685aec6 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -45,6 +45,9 @@ class WriteBufferFromHTTPServerResponse final : public BufferWithOwnMemoryonProgress(progress); }); + append_callback([&used_output, this](const Progress & progress) { + used_output.out->onProgress(progress); + auto thread_group = CurrentThread::getGroup(); + auto peak_memory_usage = thread_group->memory_tracker.getPeak(); + used_output.out->onMemoryUsage(peak_memory_usage); + }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { From b276f450f38247131bac33587482f130b4c02b4b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 28 Jun 2023 14:12:32 +0000 Subject: [PATCH 064/226] Remove whilespace --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index b8c852e2482e..562f1e7e93f3 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -816,7 +816,7 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output, this](const Progress & progress) { + append_callback([&used_output, this](const Progress & progress) { used_output.out->onProgress(progress); auto thread_group = CurrentThread::getGroup(); auto peak_memory_usage = thread_group->memory_tracker.getPeak(); From 98966796d0e003d618aade919f7f3e52788ce7e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:18:47 +0800 Subject: [PATCH 065/226] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5197b786884a..f6b629f11792 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -574,9 +574,9 @@ Alias: Like `substring` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -## substringIndex(s, delim, index) +## substringIndex(s, delim, count) -Returns the substring of `s` before `index` occurrences of the delimiter `delim`, as in Spark or MySQL. +Returns the substring of `s` before `count` occurrences of the delimiter `delim`, as in Spark or MySQL. **Syntax** From e2236384d1795ac8f95cb1281b7e5199f8844e8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:18:54 +0800 Subject: [PATCH 066/226] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f6b629f11792..f3bcc99d83d3 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -605,7 +605,7 @@ Result: └──────────────────────────────────────────────┘ ``` -## substringIndexUTF8(s, delim, index) +## substringIndexUTF8(s, delim, count) Like `substringIndex` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. From e9bac152e1a5f08845c2d40e608ed293bd5c0384 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:04 +0800 Subject: [PATCH 067/226] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f3bcc99d83d3..3ed604348348 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -581,7 +581,7 @@ Returns the substring of `s` before `count` occurrences of the delimiter `delim` **Syntax** ```sql -substringIndex(s, delim, index) +substringIndex(s, delim, count) ``` Alias: `SUBSTRING_INDEX` From 40ded2eca001ecc145358d4ab4c3a5e43738d2e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:13 +0800 Subject: [PATCH 068/226] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 3ed604348348..12aa8d2c076c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -590,7 +590,7 @@ Alias: `SUBSTRING_INDEX` - s: The string to extract substring from. [String](../../sql-reference/data-types/string.md). - delim: The character to split. [String](../../sql-reference/data-types/string.md). -- index: The number of occurrences of the delimiter to count before extracting the substring. If index is positive, everything to the left of the final delimiter (counting from the left) is returned. If index is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) +- count: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) **Example** From f049914c2f47d3e202a936464f4fd04390c1669f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:19 +0800 Subject: [PATCH 069/226] Update src/Functions/substringIndex.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Functions/substringIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index fbb20b245f6d..34d3ab4b3fbb 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -57,7 +57,7 @@ namespace arguments[1]->getName(), getName()); - if (!isNativeNumber(arguments[2])) + if (!isNativeInteger(arguments[2])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}", From 21ff69772caf01c03512c8076bcf0e95e7805588 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:28 +0800 Subject: [PATCH 070/226] Update src/Functions/substringIndex.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Functions/substringIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 34d3ab4b3fbb..fb74936b0bce 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -46,7 +46,7 @@ namespace if (!isString(arguments[0])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of first argument of function {}", + "Illegal type {} of first argument of function {}, String expected", arguments[0]->getName(), getName()); From a005b5d0c8aaefb8d1c4b74fffefe464e882d329 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:39 +0800 Subject: [PATCH 071/226] Update src/Functions/substringIndex.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Functions/substringIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index fb74936b0bce..653ef9e509af 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -53,7 +53,7 @@ namespace if (!isString(arguments[1])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of second argument of function {}", + "Illegal type {} of second argument of function {}, String expected", arguments[1]->getName(), getName()); From a35476ee13573b37ad0ae667c3c4b2405f681d01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:19:48 +0800 Subject: [PATCH 072/226] Update src/Functions/substringIndex.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Functions/substringIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 653ef9e509af..963420a4fee8 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -60,7 +60,7 @@ namespace if (!isNativeInteger(arguments[2])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of third argument of function {}", + "Illegal type {} of third argument of function {}, Integer expected", arguments[2]->getName(), getName()); From 9e34227fe96a2c37d8895663c7fecdd3688037fd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Jun 2023 10:39:20 +0800 Subject: [PATCH 073/226] change as requested --- src/Functions/substringIndex.cpp | 27 +++++++++++++++++++++------ 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 963420a4fee8..903edfe50319 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -136,8 +136,13 @@ namespace { StringRef str_ref = str_column->getDataAt(i); Int64 index = index_column->getInt(i); - StringRef res_ref - = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + + StringRef res_ref; + if constexpr (!is_utf8) + res_ref = substringIndex(str_ref, delim[0], index); + else + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -159,8 +164,13 @@ namespace for (size_t i = 0; i < rows; ++i) { StringRef str_ref = str_column->getDataAt(i); - StringRef res_ref - = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + + StringRef res_ref; + if constexpr (!is_utf8) + res_ref = substringIndex(str_ref, delim[0], index); + else + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + appendToResultColumn(res_ref, res_data, res_offsets); } } @@ -183,8 +193,13 @@ namespace for (size_t i = 0; i < rows; ++i) { Int64 index = index_column->getInt(i); - StringRef res_ref - = !is_utf8 ? substringIndex(str_ref, delim[0], index) : substringIndexUTF8(searcher.get(), str_ref, delim, index); + + StringRef res_ref; + if constexpr (!is_utf8) + res_ref = substringIndex(str_ref, delim[0], index); + else + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + appendToResultColumn(res_ref, res_data, res_offsets); } } From 3a01a859d9aaef5fc6bba54cc0a40c7c30c4a23f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 29 Jun 2023 10:39:25 +0800 Subject: [PATCH 074/226] Update src/Functions/substringIndex.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Functions/substringIndex.cpp | 44 ++++++++------------------------ 1 file changed, 11 insertions(+), 33 deletions(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 963420a4fee8..41b46af284a2 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -256,41 +256,19 @@ namespace if (index == 0) return {str_ref.data, 0}; - if (index > 0) + const auto pos = index > 0 ? str_ref.data : str_ref.data + str_ref.size - 1; + const auto end = index > 0 ? str_ref.data + str_ref.size : str_ref.data - 1; + int d = index > 0 ? 1 : -1; + + for (; index; pos += d) { - const auto * end = str_ref.data + str_ref.size; - const auto * pos = str_ref.data; - Int64 i = 0; - while (i < index) - { - pos = std::find(pos, end, delim); - if (pos != end) - { - ++pos; - ++i; - } - else - return str_ref; - } - return {str_ref.data, static_cast(pos - str_ref.data - 1)}; + if (pos == end) + return str_ref; + if (*pos == delim) + index -= d; } - else - { - const auto * begin = str_ref.data; - const auto * pos = str_ref.data + str_ref.size; - Int64 i = 0; - while (i + index < 0) - { - --pos; - while (pos >= begin && *pos != delim) - --pos; - - if (pos >= begin) - ++i; - else - return str_ref; - } - return {pos + 1, static_cast(str_ref.data + str_ref.size - pos - 1)}; + pos -= d; + return {d > 0 ? str_ref.data : pos + 1, static_cast(d > 0 ? pos - str_ref.data : str_ref.data + str_ref.size - pos - 1)} ; } } }; From 95a9270b747322bb376dbfacde8aa58ce0835930 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 29 Jun 2023 10:54:40 +0800 Subject: [PATCH 075/226] change as request --- src/Functions/substringIndex.cpp | 70 ++++++++++++++++---------------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index ade8d0ce5045..d1791c9696b7 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -71,7 +71,7 @@ namespace { ColumnPtr column_string = arguments[0].column; ColumnPtr column_delim = arguments[1].column; - ColumnPtr column_index = arguments[2].column; + ColumnPtr column_count = arguments[2].column; const ColumnConst * column_delim_const = checkAndGetColumnConst(column_delim.get()); if (!column_delim_const) @@ -97,7 +97,7 @@ namespace if (column_string_const) { String str = column_string_const->getValue(); - constantVector(str, delim, column_index.get(), vec_res, offsets_res); + constantVector(str, delim, column_count.get(), vec_res, offsets_res); } else { @@ -105,14 +105,14 @@ namespace if (!col_str) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument to {} must be a String", getName()); - bool is_index_const = isColumnConst(*column_index); - if (is_index_const) + bool is_count_const = isColumnConst(*column_count); + if (is_count_const) { - Int64 index = column_index->getInt(0); - vectorConstant(col_str, delim, index, vec_res, offsets_res); + Int64 count = column_count->getInt(0); + vectorConstant(col_str, delim, count, vec_res, offsets_res); } else - vectorVector(col_str, delim, column_index.get(), vec_res, offsets_res); + vectorVector(col_str, delim, column_count.get(), vec_res, offsets_res); } return column_res; } @@ -121,7 +121,7 @@ namespace static void vectorVector( const ColumnString * str_column, const String & delim, - const IColumn * index_column, + const IColumn * count_column, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { @@ -135,13 +135,13 @@ namespace for (size_t i = 0; i < rows; ++i) { StringRef str_ref = str_column->getDataAt(i); - Int64 index = index_column->getInt(i); + Int64 count = count_column->getInt(i); StringRef res_ref; if constexpr (!is_utf8) - res_ref = substringIndex(str_ref, delim[0], index); + res_ref = substringIndex(str_ref, delim[0], count); else - res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); appendToResultColumn(res_ref, res_data, res_offsets); } @@ -150,7 +150,7 @@ namespace static void vectorConstant( const ColumnString * str_column, const String & delim, - Int64 index, + Int64 count, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { @@ -167,9 +167,9 @@ namespace StringRef res_ref; if constexpr (!is_utf8) - res_ref = substringIndex(str_ref, delim[0], index); + res_ref = substringIndex(str_ref, delim[0], count); else - res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); appendToResultColumn(res_ref, res_data, res_offsets); } @@ -178,11 +178,11 @@ namespace static void constantVector( const String & str, const String & delim, - const IColumn * index_column, + const IColumn * count_column, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - size_t rows = index_column->size(); + size_t rows = count_column->size(); res_data.reserve(str.size() * rows / 2); res_offsets.reserve(rows); @@ -192,13 +192,13 @@ namespace StringRef str_ref{str.data(), str.size()}; for (size_t i = 0; i < rows; ++i) { - Int64 index = index_column->getInt(i); + Int64 count = count_column->getInt(i); StringRef res_ref; if constexpr (!is_utf8) - res_ref = substringIndex(str_ref, delim[0], index); + res_ref = substringIndex(str_ref, delim[0], count); else - res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, index); + res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count); appendToResultColumn(res_ref, res_data, res_offsets); } @@ -217,18 +217,18 @@ namespace } static StringRef substringIndexUTF8( - const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 index) + const PositionCaseSensitiveUTF8::SearcherInBigHaystack * searcher, const StringRef & str_ref, const String & delim, Int64 count) { - if (index == 0) + if (count == 0) return {str_ref.data, 0}; const auto * begin = reinterpret_cast(str_ref.data); const auto * end = reinterpret_cast(str_ref.data + str_ref.size); const auto * pos = begin; - if (index > 0) + if (count > 0) { Int64 i = 0; - while (i < index) + while (i < count) { pos = searcher->search(pos, end - pos); @@ -251,13 +251,13 @@ namespace ++total; } - if (total + index < 0) + if (total + count < 0) return str_ref; pos = begin; Int64 i = 0; - Int64 index_from_left = total + 1 + index; - while (i < index_from_left && pos < end && end != (pos = searcher->search(pos, end - pos))) + Int64 count_from_left = total + 1 + count; + while (i < count_from_left && pos < end && end != (pos = searcher->search(pos, end - pos))) { pos += delim.size(); ++i; @@ -266,25 +266,25 @@ namespace } } - static StringRef substringIndex(const StringRef & str_ref, char delim, Int64 index) + static StringRef substringIndex(const StringRef & str_ref, char delim, Int64 count) { - if (index == 0) + if (count == 0) return {str_ref.data, 0}; - const auto pos = index > 0 ? str_ref.data : str_ref.data + str_ref.size - 1; - const auto end = index > 0 ? str_ref.data + str_ref.size : str_ref.data - 1; - int d = index > 0 ? 1 : -1; + const auto * pos = count > 0 ? str_ref.data : str_ref.data + str_ref.size - 1; + const auto * end = count > 0 ? str_ref.data + str_ref.size : str_ref.data - 1; + int d = count > 0 ? 1 : -1; - for (; index; pos += d) + for (; count; pos += d) { if (pos == end) return str_ref; if (*pos == delim) - index -= d; + count -= d; } pos -= d; - return {d > 0 ? str_ref.data : pos + 1, static_cast(d > 0 ? pos - str_ref.data : str_ref.data + str_ref.size - pos - 1)} ; - } + return { + d > 0 ? str_ref.data : pos + 1, static_cast(d > 0 ? pos - str_ref.data : str_ref.data + str_ref.size - pos - 1)}; } }; } From ffa4f37c9af8feb2acdc4d41c4e22cf30e10986c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 29 Jun 2023 06:59:42 +0000 Subject: [PATCH 076/226] Try to fix style --- src/IO/Progress.cpp | 6 ++++-- src/Server/HTTPHandler.cpp | 6 +++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index bf42cdf91d63..c652a62574d2 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -74,7 +74,8 @@ void ProgressValues::writeJSON(WriteBuffer & out) const /// Numbers are written in double quotes (as strings) to avoid loss of precision /// of 64-bit integers after interpretation by JavaScript. - writeCString("{\"read_rows\":\"", out); + //writeCString("{\"read_rows\":\"", out); + writeCString("\"read_rows\":\"", out); writeText(read_rows, out); writeCString("\",\"read_bytes\":\"", out); writeText(read_bytes, out); @@ -88,7 +89,8 @@ void ProgressValues::writeJSON(WriteBuffer & out) const writeText(result_rows, out); writeCString("\",\"result_bytes\":\"", out); writeText(result_bytes, out); - writeCString("\"}", out); + //writeCString("\"}", out); + writeCString("\"", out); } bool Progress::incrementPiecewiseAtomically(const Progress & rhs) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 562f1e7e93f3..c7ec4a848e9a 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -816,12 +816,12 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output, this](const Progress & progress) { + append_callback([&used_output](const Progress & progress) { used_output.out->onProgress(progress); auto thread_group = CurrentThread::getGroup(); auto peak_memory_usage = thread_group->memory_tracker.getPeak(); - used_output.out->onMemoryUsage(peak_memory_usage); - }); + used_output.out->onMemoryUsage(peak_memory_usage); + }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { From 7e992955df72f27e59d0da2bd2630b9715a9a8a7 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 29 Jun 2023 08:12:33 +0000 Subject: [PATCH 077/226] Next trying slyle fix --- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 10 ++++++++-- src/Server/HTTPHandler.cpp | 7 ++++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 1e6d520de0ab..62c7b6e7346e 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -3,7 +3,7 @@ #include #include #include - +#include namespace DB { @@ -35,10 +35,16 @@ void WriteBufferFromHTTPServerResponse::writeHeaderSummary() return; WriteBufferFromOwnString progress_string_writer; + + writeCString("{", progress_string_writer); accumulated_progress.writeJSON(progress_string_writer); + writeCString(",\"peak_memory_usage\":\"", progress_string_writer); + writeText(peak_memory_usage, progress_string_writer); + writeCString("\"}", progress_string_writer); + if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << " Mem " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "\r\n" << std::flush; + *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << "\r\n" << std::flush; } void WriteBufferFromHTTPServerResponse::writeHeaderProgress() diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c7ec4a848e9a..2c4e3bacbd81 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -816,12 +816,13 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output](const Progress & progress) { + append_callback([&used_output](const Progress & progress) + { used_output.out->onProgress(progress); auto thread_group = CurrentThread::getGroup(); auto peak_memory_usage = thread_group->memory_tracker.getPeak(); - used_output.out->onMemoryUsage(peak_memory_usage); - }); + used_output.out->onMemoryUsage(peak_memory_usage); + }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { From 6f4d6fd8d6e54b947e7f4634f308354694926715 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 29 Jun 2023 08:29:16 +0000 Subject: [PATCH 078/226] Remove whitespace --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 2c4e3bacbd81..e3ff43ba41ad 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -816,7 +816,7 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` - append_callback([&used_output](const Progress & progress) + append_callback([&used_output](const Progress & progress) { used_output.out->onProgress(progress); auto thread_group = CurrentThread::getGroup(); From f88c1fdf44e4f1a6da7bf0db743416f4c020718f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Jun 2023 11:59:09 +0000 Subject: [PATCH 079/226] Fixes --- src/Storages/HDFS/HDFSCommon.h | 18 ++++++++++++++++-- src/Storages/StorageAzureBlob.cpp | 2 +- src/Storages/StorageAzureBlob.h | 12 +++--------- src/Storages/StorageFile.cpp | 6 ++++-- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index 5eb687695f29..23f9e4d8f129 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -58,8 +58,22 @@ static const String CONFIG_PREFIX; HDFSBuilderWrapper(const HDFSBuilderWrapper &) = delete; HDFSBuilderWrapper & operator=(const HDFSBuilderWrapper &) = delete; - HDFSBuilderWrapper(HDFSBuilderWrapper &&) = default; - HDFSBuilderWrapper & operator=(HDFSBuilderWrapper &&) = default; + + HDFSBuilderWrapper(HDFSBuilderWrapper && other) noexcept + { + *this = std::move(other); + } + + HDFSBuilderWrapper & operator=(HDFSBuilderWrapper && other) noexcept + { + std::swap(hdfs_builder, other.hdfs_builder); + config_stor = std::move(other.config_stor); + hadoop_kerberos_keytab = std::move(other.hadoop_kerberos_keytab); + hadoop_kerberos_principal = std::move(other.hadoop_kerberos_principal); + hadoop_security_kerberos_ticket_cache_path = std::move(other.hadoop_security_kerberos_ticket_cache_path); + need_kinit = std::move(other.need_kinit); + return *this; + } hdfsBuilder * get() { return hdfs_builder; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 95ce8ce35525..365de2611ce0 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -940,7 +940,7 @@ void StorageAzureBlobSource::GlobIterator::createFilterAST(const String & any_ke StorageAzureBlobSource::KeysIterator::KeysIterator( AzureObjectStorage * object_storage_, const std::string & container_, - Strings keys_, + const Strings & keys_, ASTPtr query_, const Block & virtual_header_, ContextPtr context_, diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index fdd4bd1afb73..ad87da1f61aa 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -148,7 +148,6 @@ class StorageAzureBlobSource : public ISource, WithContext IIterator(ContextPtr context_):WithContext(context_) {} virtual ~IIterator() = default; virtual RelativePathWithMetadata next() = 0; - virtual size_t getTotalSize() const = 0; RelativePathWithMetadata operator ()() { return next(); } }; @@ -167,7 +166,6 @@ class StorageAzureBlobSource : public ISource, WithContext std::function file_progress_callback_ = {}); RelativePathWithMetadata next() override; - size_t getTotalSize() const override; ~GlobIterator() override = default; private: @@ -179,7 +177,6 @@ class StorageAzureBlobSource : public ISource, WithContext Block virtual_header; size_t index = 0; - std::atomic total_size = 0; RelativePathsWithMetadata blobs_with_metadata; RelativePathsWithMetadata * outer_blobs; @@ -202,14 +199,14 @@ class StorageAzureBlobSource : public ISource, WithContext KeysIterator( AzureObjectStorage * object_storage_, const std::string & container_, - Strings keys_, + const Strings & keys_, ASTPtr query_, const Block & virtual_header_, ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_); + RelativePathsWithMetadata * outer_blobs, + std::function file_progress_callback = {}); RelativePathWithMetadata next() override; - size_t getTotalSize() const override; ~KeysIterator() override = default; private: @@ -222,9 +219,6 @@ class StorageAzureBlobSource : public ISource, WithContext Block virtual_header; std::atomic index = 0; - std::atomic total_size = 0; - - RelativePathsWithMetadata * outer_blobs; }; StorageAzureBlobSource( diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e9faa5e112a1..d87aac50deb0 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -739,8 +739,10 @@ class StorageFileSource : public ISource if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - size_t chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size); + size_t chunk_size = 0; + if (storage->format_name != "Distributed") + chunk_size = input_format->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); /// Enrich with virtual columns. if (files_info->need_path_column) From bdfaffb6e241d8507c3b127f0288d09125ae0fb0 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 29 Jun 2023 12:08:08 +0000 Subject: [PATCH 080/226] Add/fix tests --- src/IO/Progress.cpp | 12 +++++++----- src/IO/Progress.h | 4 ++-- .../HTTP/WriteBufferFromHTTPServerResponse.cpp | 3 +-- .../00416_pocopatch_progress_in_http_headers.sh | 2 +- tests/queries/0_stateless/01921_test_progress_bar.py | 7 ++++--- tests/queries/0_stateless/02136_scalar_progress.sh | 2 +- .../0_stateless/02373_progress_contain_result.sh | 2 +- .../0_stateless/02423_insert_summary_behaviour.sh | 12 ++++++------ .../0_stateless/02457_insert_select_progress_http.sh | 2 +- 9 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index c652a62574d2..6a78a453d8a2 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -69,12 +69,13 @@ void ProgressValues::write(WriteBuffer & out, UInt64 client_revision) const } } -void ProgressValues::writeJSON(WriteBuffer & out) const +void ProgressValues::writeJSON(WriteBuffer & out, bool add_braces) const { /// Numbers are written in double quotes (as strings) to avoid loss of precision /// of 64-bit integers after interpretation by JavaScript. - //writeCString("{\"read_rows\":\"", out); + if (add_braces) + writeCString("{", out); writeCString("\"read_rows\":\"", out); writeText(read_rows, out); writeCString("\",\"read_bytes\":\"", out); @@ -89,8 +90,9 @@ void ProgressValues::writeJSON(WriteBuffer & out) const writeText(result_rows, out); writeCString("\",\"result_bytes\":\"", out); writeText(result_bytes, out); - //writeCString("\"}", out); writeCString("\"", out); + if (add_braces) + writeCString("}", out); } bool Progress::incrementPiecewiseAtomically(const Progress & rhs) @@ -232,9 +234,9 @@ void Progress::write(WriteBuffer & out, UInt64 client_revision) const getValues().write(out, client_revision); } -void Progress::writeJSON(WriteBuffer & out) const +void Progress::writeJSON(WriteBuffer & out, bool add_braces) const { - getValues().writeJSON(out); + getValues().writeJSON(out, add_braces); } } diff --git a/src/IO/Progress.h b/src/IO/Progress.h index c21b1b854b0b..7cf67a0a4321 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -32,7 +32,7 @@ struct ProgressValues void read(ReadBuffer & in, UInt64 server_revision); void write(WriteBuffer & out, UInt64 client_revision) const; - void writeJSON(WriteBuffer & out) const; + void writeJSON(WriteBuffer & out, bool add_braces = true) const; }; struct ReadProgress @@ -118,7 +118,7 @@ struct Progress void write(WriteBuffer & out, UInt64 client_revision) const; /// Progress in JSON format (single line, without whitespaces) is used in HTTP headers. - void writeJSON(WriteBuffer & out) const; + void writeJSON(WriteBuffer & out, bool add_braces = true) const; /// Each value separately is changed atomically (but not whole object). bool incrementPiecewiseAtomically(const Progress & rhs); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 62c7b6e7346e..544442c4c05d 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -37,11 +37,10 @@ void WriteBufferFromHTTPServerResponse::writeHeaderSummary() WriteBufferFromOwnString progress_string_writer; writeCString("{", progress_string_writer); - accumulated_progress.writeJSON(progress_string_writer); + accumulated_progress.writeJSON(progress_string_writer, false); writeCString(",\"peak_memory_usage\":\"", progress_string_writer); writeText(peak_memory_usage, progress_string_writer); writeCString("\"}", progress_string_writer); - if (response_header_ostr) *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << "\r\n" << std::flush; diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index b2189ab0cc2a..d830debdf55c 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -26,7 +26,7 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'DROP ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'CREATE TABLE insert_number_query (record UInt32) Engine = Memory' > /dev/null 2>&1 ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'CREATE TABLE insert_number_query_2 (record UInt32) Engine = Memory' > /dev/null 2>&1 -${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&http_headers_progress_interval_ms=0&send_progress_in_http_headers=1" -d 'INSERT INTO insert_number_query (record) SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Summary|^[0-9]' +${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&http_headers_progress_interval_ms=0&send_progress_in_http_headers=1" -d 'INSERT INTO insert_number_query (record) SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Summary|^[0-9]' | sed 's/,\"peak_mem[^}]*//' ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'DROP TABLE insert_number_query' > /dev/null 2>&1 ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'DROP TABLE insert_number_query_2' > /dev/null 2>&1 diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 3b0b429d3966..9ce2168e2ae2 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -14,6 +14,7 @@ with client(name="client1>", log=log) as client1: client1.expect(prompt) - client1.send("SELECT number FROM numbers(100) FORMAT Null") - client1.expect("Progress: 100\.00 rows, 800\.00 B.*" + end_of_block) - client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec." + end_of_block) + client1.send("SELECT number FROM numbers(1000) FORMAT Null") + client1.expect("Progress: 1\.00 thousand rows, 8\.00 KB .*" + end_of_block) + client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec.") + client1.expect("Peak memory usage \(for query\) .*B" + end_of_block) diff --git a/tests/queries/0_stateless/02136_scalar_progress.sh b/tests/queries/0_stateless/02136_scalar_progress.sh index 9f4429b0caaf..517fe19ecd47 100755 --- a/tests/queries/0_stateless/02136_scalar_progress.sh +++ b/tests/queries/0_stateless/02136_scalar_progress.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" +$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" | sed 's/,\"peak_mem[^}]*//' diff --git a/tests/queries/0_stateless/02373_progress_contain_result.sh b/tests/queries/0_stateless/02373_progress_contain_result.sh index 1b257b699f59..c84af0ee2696 100755 --- a/tests/queries/0_stateless/02373_progress_contain_result.sh +++ b/tests/queries/0_stateless/02373_progress_contain_result.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo 'SELECT 1 FROM numbers(100)' | ${CLICKHOUSE_CURL_COMMAND} -v "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0" --data-binary @- 2>&1 | - grep 'X-ClickHouse-Summary' + grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' diff --git a/tests/queries/0_stateless/02423_insert_summary_behaviour.sh b/tests/queries/0_stateless/02423_insert_summary_behaviour.sh index 6c75efa5150c..a0b3c5198065 100755 --- a/tests/queries/0_stateless/02423_insert_summary_behaviour.sh +++ b/tests/queries/0_stateless/02423_insert_summary_behaviour.sh @@ -11,11 +11,11 @@ $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target TO target_1 AS $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target_2 TO target_2 AS SELECT * FROM floats, numbers(2) n" echo "No materialized views" -${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' echo "With materialized views" -${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' diff --git a/tests/queries/0_stateless/02457_insert_select_progress_http.sh b/tests/queries/0_stateless/02457_insert_select_progress_http.sh index 656ab3dc403c..7f7fc67ae00b 100755 --- a/tests/queries/0_stateless/02457_insert_select_progress_http.sh +++ b/tests/queries/0_stateless/02457_insert_select_progress_http.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "insert into function null('_ Int') select * from numbers(5) settings max_block_size=1" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: + grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | sed 's/,\"peak_mem[^}]*//' } From 795e34fcf645d9ac200933ae3be78406972f28e5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Jun 2023 12:36:50 +0000 Subject: [PATCH 081/226] Fix test --- tests/integration/test_storage_s3/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 6c251d2f84e8..619157c83690 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1361,16 +1361,16 @@ def test_select_columns(started_cluster): instance.query("SYSTEM FLUSH LOGS") result1 = instance.query( - f"SELECT read_bytes FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT value2 FROM {name}'" + f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT value2 FROM {name}'" ) instance.query(f"SELECT * FROM {name}") instance.query("SYSTEM FLUSH LOGS") result2 = instance.query( - f"SELECT read_bytes FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT * FROM {name}'" + f"SELECT ProfileEvents['ReadBufferFromS3Bytes'] FROM system.query_log WHERE type='QueryFinish' and query LIKE 'SELECT * FROM {name}'" ) - assert int(result1) * 3 <= int(result2) + assert round(int(result2) / int(result1)) == 3 def test_insert_select_schema_inference(started_cluster): From 54838e92d48d9f0f177a48c85f633a9a83a13097 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 29 Jun 2023 13:20:38 +0000 Subject: [PATCH 082/226] Remove whitespace --- src/IO/Progress.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index 6a78a453d8a2..620d2f0f7622 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -69,7 +69,7 @@ void ProgressValues::write(WriteBuffer & out, UInt64 client_revision) const } } -void ProgressValues::writeJSON(WriteBuffer & out, bool add_braces) const +void ProgressValues::writeJSON(WriteBuffer & out, bool add_braces) const { /// Numbers are written in double quotes (as strings) to avoid loss of precision /// of 64-bit integers after interpretation by JavaScript. From 901089bfab56e3d3cd905abf671ca1d774634a4d Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 30 Jun 2023 05:15:41 +0000 Subject: [PATCH 083/226] Remove comment --- src/Common/ProgressIndication.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 29766fd1a148..c9068720f8fb 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -166,8 +166,6 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) if (memory_usage > 0) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM"; - // profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << ", " - // << formatReadableSizeWithDecimalSuffix(peak_usage) << " Peak RAM"; if (max_host_usage < memory_usage) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(max_host_usage) << " max/host"; From 289d9849d408d9bd38e95d89b6434a8d6bf57664 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 22:50:09 +0200 Subject: [PATCH 084/226] Allow SQL standard FETCH without OFFSET --- src/Parsers/ParserSelectQuery.cpp | 92 ++++++++++--------- .../02790_sql_standard_fetch.reference | 36 ++++++++ .../0_stateless/02790_sql_standard_fetch.sql | 31 +++++++ 3 files changed, 118 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/02790_sql_standard_fetch.reference create mode 100644 tests/queries/0_stateless/02790_sql_standard_fetch.sql diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 1c48f7738234..341c1ef60b49 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -292,6 +292,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// This is needed for TOP expression, because it can also use WITH TIES. bool limit_with_ties_occured = false; + bool has_offset_clause = false; + bool offset_clause_has_sql_standard_row_or_rows = false; /// OFFSET offset_row_count {ROW | ROWS} + /// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list | LIMIT offset, length BY expr-list if (s_limit.ignore(pos, expected)) { @@ -316,6 +319,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!exp_elem.parse(pos, limit_offset, expected)) return false; + + has_offset_clause = true; } else if (s_with_ties.ignore(pos, expected)) { @@ -351,60 +356,65 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (s_offset.ignore(pos, expected)) { - /// OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES} - bool offset_with_fetch_maybe = false; + /// OFFSET without LIMIT + + has_offset_clause = true; if (!exp_elem.parse(pos, limit_offset, expected)) return false; + /// SQL standard OFFSET N ROW[S] ... + if (s_row.ignore(pos, expected)) + offset_clause_has_sql_standard_row_or_rows = true; + + if (s_rows.ignore(pos, expected)) { - if (s_rows.ignore(pos, expected)) + if (offset_clause_has_sql_standard_row_or_rows) throw Exception(ErrorCodes::ROW_AND_ROWS_TOGETHER, "Can not use ROW and ROWS together"); - offset_with_fetch_maybe = true; - } - else if (s_rows.ignore(pos, expected)) - { - offset_with_fetch_maybe = true; + + offset_clause_has_sql_standard_row_or_rows = true; } + } - if (offset_with_fetch_maybe && s_fetch.ignore(pos, expected)) - { - /// OFFSET FETCH clause must exists with "ORDER BY" - if (!order_expression_list) - throw Exception(ErrorCodes::OFFSET_FETCH_WITHOUT_ORDER_BY, "Can not use OFFSET FETCH clause without ORDER BY"); + /// SQL standard FETCH (either following SQL standard OFFSET or following ORDER BY) + if ((!has_offset_clause || offset_clause_has_sql_standard_row_or_rows) + && s_fetch.ignore(pos, expected)) + { + /// FETCH clause must exist with "ORDER BY" + if (!order_expression_list) + throw Exception(ErrorCodes::OFFSET_FETCH_WITHOUT_ORDER_BY, "Can not use OFFSET FETCH clause without ORDER BY"); - if (s_first.ignore(pos, expected)) - { - if (s_next.ignore(pos, expected)) - throw Exception(ErrorCodes::FIRST_AND_NEXT_TOGETHER, "Can not use FIRST and NEXT together"); - } - else if (!s_next.ignore(pos, expected)) - return false; + if (s_first.ignore(pos, expected)) + { + if (s_next.ignore(pos, expected)) + throw Exception(ErrorCodes::FIRST_AND_NEXT_TOGETHER, "Can not use FIRST and NEXT together"); + } + else if (!s_next.ignore(pos, expected)) + return false; - if (!exp_elem.parse(pos, limit_length, expected)) - return false; + if (!exp_elem.parse(pos, limit_length, expected)) + return false; - if (s_row.ignore(pos, expected)) - { - if (s_rows.ignore(pos, expected)) - throw Exception(ErrorCodes::ROW_AND_ROWS_TOGETHER, "Can not use ROW and ROWS together"); - } - else if (!s_rows.ignore(pos, expected)) - return false; + if (s_row.ignore(pos, expected)) + { + if (s_rows.ignore(pos, expected)) + throw Exception(ErrorCodes::ROW_AND_ROWS_TOGETHER, "Can not use ROW and ROWS together"); + } + else if (!s_rows.ignore(pos, expected)) + return false; - if (s_with_ties.ignore(pos, expected)) - { - select_query->limit_with_ties = true; - } - else if (s_only.ignore(pos, expected)) - { - select_query->limit_with_ties = false; - } - else - { - return false; - } + if (s_with_ties.ignore(pos, expected)) + { + select_query->limit_with_ties = true; + } + else if (s_only.ignore(pos, expected)) + { + select_query->limit_with_ties = false; + } + else + { + return false; } } diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference new file mode 100644 index 000000000000..429eecbc9364 --- /dev/null +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -0,0 +1,36 @@ +┌─id─┬─name──┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry │ it │ 104 │ +│ 24 │ Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave │ sales │ 96 │ +└────┴───────┴────────────┴────────┘ +┌─id─┬─name──┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry │ it │ 104 │ +│ 24 │ Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave │ sales │ 96 │ +└────┴───────┴────────────┴────────┘ +┌─id─┬─name──┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry │ it │ 104 │ +│ 24 │ Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Cindy │ sales │ 96 │ +│ 32 │ Dave │ sales │ 96 │ +└────┴───────┴────────────┴────────┘ +┌─id─┬─name──┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Cindy │ sales │ 96 │ +│ 32 │ Dave │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────┴────────────┴────────┘ +┌─id─┬─name──┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Cindy │ sales │ 96 │ +│ 32 │ Dave │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql new file mode 100644 index 000000000000..58ffa035d47e --- /dev/null +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -0,0 +1,31 @@ +# https://antonz.org/sql-fetch/ + +CREATE TEMPORARY TABLE employees (id UInt64, name String, department String, salary UInt64); +INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', '84'); + +select * from employees +order by salary desc +limit 5 +format PrettyCompactNoEscapes; + +select * from employees +order by salary desc +fetch first 5 rows only +format PrettyCompactNoEscapes; + +select * from employees +order by salary desc +fetch first 5 rows with ties +format PrettyCompactNoEscapes; + +select * from employees +order by salary desc +offset 3 rows +fetch next 5 rows only +format PrettyCompactNoEscapes; + +select * from employees +order by salary desc +offset 3 rows +fetch first 5 rows only +format PrettyCompactNoEscapes; From caa75a7fc38ab95a405488b3826022683160eed9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 20:59:28 +0200 Subject: [PATCH 085/226] Make the test stable --- .../0_stateless/02790_sql_standard_fetch.sql | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql index 58ffa035d47e..4204279a7461 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.sql +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -1,30 +1,33 @@ -# https://antonz.org/sql-fetch/ +-- https://antonz.org/sql-fetch/ CREATE TEMPORARY TABLE employees (id UInt64, name String, department String, salary UInt64); INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', '84'); -select * from employees +-- Determinism +SET max_threads = 1, parallelize_output_from_storages = 0; + +select * from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc limit 5 format PrettyCompactNoEscapes; -select * from employees +select * from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows only format PrettyCompactNoEscapes; -select * from employees +select * from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows with ties format PrettyCompactNoEscapes; -select * from employees +select * from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch next 5 rows only format PrettyCompactNoEscapes; -select * from employees +select * from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch first 5 rows only From 8551a38e73df8fb4ee29ebbe0825e2c57468a70b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 21:19:30 +0200 Subject: [PATCH 086/226] Update some tests --- .../test_on_cluster_timeouts/configs/users_config.xml | 1 + tests/integration/test_quorum_inserts_parallel/test.py | 2 +- .../integration/test_read_temporary_tables_on_failure/test.py | 2 +- tests/integration/test_rename_column/test.py | 4 ++-- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_on_cluster_timeouts/configs/users_config.xml b/tests/integration/test_on_cluster_timeouts/configs/users_config.xml index 76f01279ce4d..85bd9b7215ae 100644 --- a/tests/integration/test_on_cluster_timeouts/configs/users_config.xml +++ b/tests/integration/test_on_cluster_timeouts/configs/users_config.xml @@ -2,6 +2,7 @@ 2 + 0 diff --git a/tests/integration/test_quorum_inserts_parallel/test.py b/tests/integration/test_quorum_inserts_parallel/test.py index 99548e37a548..dabd0e5ee43d 100644 --- a/tests/integration/test_quorum_inserts_parallel/test.py +++ b/tests/integration/test_quorum_inserts_parallel/test.py @@ -27,7 +27,7 @@ def started_cluster(): def test_parallel_quorum_actually_parallel(started_cluster): - settings = {"insert_quorum": "3", "insert_quorum_parallel": "1"} + settings = {"insert_quorum": "3", "insert_quorum_parallel": "1", "function_sleep_max_microseconds_per_block": "0"} for i, node in enumerate([node1, node2, node3]): node.query( "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '{num}') ORDER BY tuple()".format( diff --git a/tests/integration/test_read_temporary_tables_on_failure/test.py b/tests/integration/test_read_temporary_tables_on_failure/test.py index fd1d92eff92f..b137ebc8c941 100644 --- a/tests/integration/test_read_temporary_tables_on_failure/test.py +++ b/tests/integration/test_read_temporary_tables_on_failure/test.py @@ -19,7 +19,7 @@ def start_cluster(): def test_different_versions(start_cluster): with pytest.raises(QueryTimeoutExceedException): - node.query("SELECT sleepEachRow(3) FROM numbers(10)", timeout=5) + node.query("SELECT sleepEachRow(3) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 0", timeout=5) with pytest.raises(QueryRuntimeException): node.query("SELECT 1", settings={"max_concurrent_queries_for_user": 1}) assert node.contains_in_log("Too many simultaneous queries for user") diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index 8dc57cf08ff3..1c87b101b11a 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -159,7 +159,7 @@ def insert( ) elif slow: query.append( - "INSERT INTO {table_name} ({col0}, {col1}) SELECT number + sleepEachRow(0.001) AS {col0}, number + 1 AS {col1} FROM numbers_mt({chunk})".format( + "INSERT INTO {table_name} ({col0}, {col1}) SELECT number + sleepEachRow(0.001) AS {col0}, number + 1 AS {col1} FROM numbers_mt({chunk}) SETTINGS function_sleep_max_microseconds_per_block = 0".format( table_name=table_name, chunk=chunk, col0=col_names[0], @@ -198,7 +198,7 @@ def select( try: if slow: r = node.query( - "SELECT count() FROM (SELECT num2, sleepEachRow(0.5) FROM {} WHERE {} % 1000 > 0)".format( + "SELECT count() FROM (SELECT num2, sleepEachRow(0.5) FROM {} WHERE {} % 1000 > 0) SETTINGS function_sleep_max_microseconds_per_block = 0".format( table_name, col_name ) ) From 13cc329bfbc0f8e7506f082ba7a02218fc09f70d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 4 Jul 2023 19:32:43 +0000 Subject: [PATCH 087/226] Automatic style fix --- tests/integration/test_quorum_inserts_parallel/test.py | 6 +++++- .../test_read_temporary_tables_on_failure/test.py | 5 ++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_quorum_inserts_parallel/test.py b/tests/integration/test_quorum_inserts_parallel/test.py index dabd0e5ee43d..7f8784d822cb 100644 --- a/tests/integration/test_quorum_inserts_parallel/test.py +++ b/tests/integration/test_quorum_inserts_parallel/test.py @@ -27,7 +27,11 @@ def started_cluster(): def test_parallel_quorum_actually_parallel(started_cluster): - settings = {"insert_quorum": "3", "insert_quorum_parallel": "1", "function_sleep_max_microseconds_per_block": "0"} + settings = { + "insert_quorum": "3", + "insert_quorum_parallel": "1", + "function_sleep_max_microseconds_per_block": "0", + } for i, node in enumerate([node1, node2, node3]): node.query( "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '{num}') ORDER BY tuple()".format( diff --git a/tests/integration/test_read_temporary_tables_on_failure/test.py b/tests/integration/test_read_temporary_tables_on_failure/test.py index b137ebc8c941..77c8f3cf26b9 100644 --- a/tests/integration/test_read_temporary_tables_on_failure/test.py +++ b/tests/integration/test_read_temporary_tables_on_failure/test.py @@ -19,7 +19,10 @@ def start_cluster(): def test_different_versions(start_cluster): with pytest.raises(QueryTimeoutExceedException): - node.query("SELECT sleepEachRow(3) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 0", timeout=5) + node.query( + "SELECT sleepEachRow(3) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 0", + timeout=5, + ) with pytest.raises(QueryRuntimeException): node.query("SELECT 1", settings={"max_concurrent_queries_for_user": 1}) assert node.contains_in_log("Too many simultaneous queries for user") From 1ea526101242c4c0c64e6b0892b7fa2985f7f684 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 4 Jul 2023 22:19:53 +0200 Subject: [PATCH 088/226] Intermediate version --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 105 +++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 17 ++- 3 files changed, 123 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 23bbc1c7f9d0..7424a2484917 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -203,6 +203,8 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write sendPartFromMemory(part, out, send_projections); else sendPartFromDisk(part, out, client_protocol_version, false, send_projections); + + data.addLastSentPart(part->name); } catch (const NetException &) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dac9e6923a57..7f282b6c0e67 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3928,6 +3928,111 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam return {}; } +void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) +{ + { + std::lock_guard lock(last_sent_parts_mutex); + last_sent_parts.emplace_back(info); + while (last_sent_parts.size() > LAST_SENT_PARS_WINDOW_SIZE) + last_sent_parts.pop_front(); + } + + last_sent_parts_cv.notify_all(); +} + +void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms) +{ + if (wait_ms == 0) + { + LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because wait time is zero"); + return; + } + + auto zookeeper = getZooKeeper(); + + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper); + if (unique_parts_set.empty()) + { + LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); + return; + } + + auto wait_predicate = [&] () -> void + { + bool all_fetched = true; + for (const auto & part : unique_parts_set) + { + bool found = false; + for (const auto & sent_part : last_sent_parts) + { + if (sent_part.contains(part)) + { + found = true; + break; + } + } + if (!found) + { + all_fetched = false; + break; + } + } + return all_fetched; + }; + + std::unique_lock lock(last_sent_parts_mutex); + if (!last_sent_parts_cv.wait_for(last_sent_parts_cv, std::chrono::duration_cast(wait_ms), wait_predicate)) + LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unqiue_parts_set.size()); +} + +std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_) +{ + if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) + return {}; + + Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); + Strings our_parts; + std::vector data_parts_on_replicas; + for (const String & replica : replicas) + { + if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + continue; + + Strings parts = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas" / replica / "parts"); + if (replica == replica_name_) + { + our_parts = parts; + } + else + { + data_parts_on_replicas.emplace_back(format_version_); + for (const auto & part : parts) + { + if (!data_parts_on_replicas.back().getContainingPart(part).empty()) + data_parts_on_replicas.back().add(part); + } + } + } + + NameSet our_unique_parts; + for (const auto & part : our_parts) + { + bool found = false; + for (const auto & active_parts_set : data_parts_on_replicas) + { + if (!active_parts_set.getContainingPart(part).empty()) + { + found = true; + break; + } + } + if (!found) + our_unique_parts.insert(MergeTreePartInfo::fromPartName(part, format_version)); + } + + return our_unique_parts; +} + String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entry, bool active) { auto zookeeper = getZooKeeper(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bdd3f0da5bff..4661f0a56daa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -340,6 +340,15 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; + void addLastSentPart(const MergeTreePartInfo & info); + std::deque getLastSentParts() const + { + std::lock_guard lock(last_sent_parts_mutex); + return last_sent_parts; + } + + void waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms); + private: std::atomic_bool are_restoring_replica {false}; @@ -444,9 +453,14 @@ class StorageReplicatedMergeTree final : public MergeTreeData Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET std::atomic shutdown_called {false}; - std::atomic flush_called {false}; + + static constexpr size_t LAST_SENT_PARS_WINDOW_SIZE = 1000; + std::mutex last_sent_parts_mutex; + std::condition_variable last_sent_parts_cv; + std::deque last_sent_parts; /// Threads. + /// /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. bool queue_update_in_progress = false; @@ -697,6 +711,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData */ String findReplicaHavingCoveringPart(LogEntry & entry, bool active); String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name); + static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. From 2460268e3c260254021902f57e0e21e40d8d9d29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 23:22:08 +0200 Subject: [PATCH 089/226] Remove templates --- src/Functions/GregorianDate.cpp | 272 ++++++++++++++ src/Functions/GregorianDate.h | 453 +++++------------------- src/Functions/fromModifiedJulianDay.cpp | 5 +- src/Functions/toModifiedJulianDay.cpp | 8 +- 4 files changed, 362 insertions(+), 376 deletions(-) create mode 100644 src/Functions/GregorianDate.cpp diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp new file mode 100644 index 000000000000..0f8a95ff3e74 --- /dev/null +++ b/src/Functions/GregorianDate.cpp @@ -0,0 +1,272 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_FORMAT_DATETIME; + extern const int LOGICAL_ERROR; +} + +namespace gd +{ + static inline constexpr bool is_leap_year(int32_t year) + { + return (year % 4 == 0) && ((year % 400 == 0) || (year % 100 != 0)); + } + + static inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) + { + switch (month) + { + case 1: return 31; + case 2: return is_leap_year ? 29 : 28; + case 3: return 31; + case 4: return 30; + case 5: return 31; + case 6: return 30; + case 7: return 31; + case 8: return 31; + case 9: return 30; + case 10: return 31; + case 11: return 30; + case 12: return 31; + default: + std::terminate(); + } + } + + /** Integer division truncated toward negative infinity. + */ + template + static inline constexpr I div(I x, J y) + { + const auto y_cast = static_cast(y); + if (x > 0 && y_cast < 0) + return ((x - 1) / y_cast) - 1; + else if (x < 0 && y_cast > 0) + return ((x + 1) / y_cast) - 1; + else + return x / y_cast; + } + + /** Integer modulus, satisfying div(x, y)*y + mod(x, y) == x. + */ + template + static inline constexpr I mod(I x, J y) + { + const auto y_cast = static_cast(y); + const auto r = x % y_cast; + if ((x > 0 && y_cast < 0) || (x < 0 && y_cast > 0)) + return r == 0 ? static_cast(0) : r + y_cast; + else + return r; + } + + /** Like std::min(), but the type of operands may differ. + */ + template + static inline constexpr I min(I x, J y) + { + const auto y_cast = static_cast(y); + return x < y_cast ? x : y_cast; + } + + static inline char readDigit(ReadBuffer & in) + { + char c; + if (!in.read(c)) + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected a digit at the end of stream"); + else if (c < '0' || c > '9') + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot read input: expected a digit but got something else"); + else + return c - '0'; + } +} + +GregorianDate::GregorianDate(ReadBuffer & in) +{ + year_ = gd::readDigit(in) * 1000 + + gd::readDigit(in) * 100 + + gd::readDigit(in) * 10 + + gd::readDigit(in); + + assertChar('-', in); + + month_ = gd::readDigit(in) * 10 + + gd::readDigit(in); + + assertChar('-', in); + + day_of_month_ = gd::readDigit(in) * 10 + + gd::readDigit(in); + + assertEOF(in); + + if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > gd::monthLength(gd::is_leap_year(year_), month_)) + throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date: {}", toString()); +} + +GregorianDate::GregorianDate(int64_t modified_julian_day) +{ + const OrdinalDate ord(modified_julian_day); + const MonthDay md(gd::is_leap_year(ord.year()), ord.dayOfYear()); + + year_ = ord.year(); + month_ = md.month(); + day_of_month_ = md.dayOfMonth(); +} + +int64_t GregorianDate::toModifiedJulianDay() const +{ + const MonthDay md(month_, day_of_month_); + const auto day_of_year = md.dayOfYear(gd::is_leap_year(year_)); + const OrdinalDate ord(year_, day_of_year); + return ord.toModifiedJulianDay(); +} + +template +ReturnType GregorianDate::writeImpl(WriteBuffer & buf) const +{ + if (year_ < 0 || year_ > 9999) + { + if constexpr (std::is_same_v) + throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME, + "Impossible to stringify: year too big or small: {}", DB::toString(year_)); + else + return false; + } + else + { + auto y = year_; + writeChar('0' + y / 1000, buf); y %= 1000; + writeChar('0' + y / 100, buf); y %= 100; + writeChar('0' + y / 10, buf); y %= 10; + writeChar('0' + y , buf); + + writeChar('-', buf); + + auto m = month_; + writeChar('0' + m / 10, buf); m %= 10; + writeChar('0' + m , buf); + + writeChar('-', buf); + + auto d = day_of_month_; + writeChar('0' + d / 10, buf); d %= 10; + writeChar('0' + d , buf); + } + + return ReturnType(true); +} + +std::string GregorianDate::toString() const +{ + WriteBufferFromOwnString buf; + write(buf); + return buf.str(); +} + +OrdinalDate::OrdinalDate(int32_t year, uint16_t day_of_year) + : year_(year) + , day_of_year_(day_of_year) +{ + if (day_of_year < 1 || day_of_year > (gd::is_leap_year(year) ? 366 : 365)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid ordinal date: {}-{}", toString(year), toString(day_of_year)); + } +} + +OrdinalDate::OrdinalDate(int64_t modified_julian_day) +{ + /// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively). + + if (modified_julian_day < -678941) + throw Exception( + ErrorCodes::CANNOT_FORMAT_DATETIME, + "Value cannot be represented as date because it's out of range"); + + if (modified_julian_day > 2973119) + throw Exception( + ErrorCodes::CANNOT_FORMAT_DATETIME, + "Value cannot be represented as date because it's out of range"); + + const auto a = modified_julian_day + 678575; + const auto quad_cent = gd::div(a, 146097); + const auto b = gd::mod(a, 146097); + const auto cent = gd::min(gd::div(b, 36524), 3); + const auto c = b - cent * 36524; + const auto quad = gd::div(c, 1461); + const auto d = gd::mod(c, 1461); + const auto y = gd::min(gd::div(d, 365), 3); + + day_of_year_ = d - y * 365 + 1; + year_ = static_cast(quad_cent * 400 + cent * 100 + quad * 4 + y + 1); +} + +int64_t OrdinalDate::toModifiedJulianDay() const noexcept +{ + const auto y = year_ - 1; + return day_of_year_ + + 365 * y + + gd::div(y, 4) + - gd::div(y, 100) + + gd::div(y, 400) + - 678576; +} + +MonthDay::MonthDay(uint8_t month, uint8_t day_of_month) + : month_(month) + , day_of_month_(day_of_month) +{ + if (month < 1 || month > 12) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid month: {}", DB::toString(month)); + /* We can't validate day_of_month here, because we don't know if + * it's a leap year. */ +} + +MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) +{ + if (day_of_year < 1 || day_of_year > (is_leap_year ? 366 : 365)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year: {}{}", + (is_leap_year ? "leap, " : "non-leap, "), DB::toString(day_of_year)); + + month_ = 1; + uint16_t d = day_of_year; + while (true) + { + const auto len = gd::monthLength(is_leap_year, month_); + if (d <= len) + break; + month_++; + d -= len; + } + day_of_month_ = d; +} + +uint16_t MonthDay::dayOfYear(bool is_leap_year) const +{ + if (day_of_month_ < 1 || day_of_month_ > gd::monthLength(is_leap_year, month_)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month: {}{}-{}", + (is_leap_year ? "leap, " : "non-leap, "), DB::toString(month_), DB::toString(day_of_month_)); + } + const auto k = month_ <= 2 ? 0 : is_leap_year ? -1 :-2; + return (367 * month_ - 362) / 12 + k + day_of_month_; +} + +template void GregorianDate::writeImpl(WriteBuffer & buf) const; +template bool GregorianDate::writeImpl(WriteBuffer & buf) const; + +} diff --git a/src/Functions/GregorianDate.h b/src/Functions/GregorianDate.h index 16fcb5ea0619..4a0cbec5afe6 100644 --- a/src/Functions/GregorianDate.h +++ b/src/Functions/GregorianDate.h @@ -1,425 +1,138 @@ #pragma once -#include -#include #include -#include -#include -#include -#include - -#include namespace DB { - namespace ErrorCodes - { - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_FORMAT_DATETIME; - extern const int LOGICAL_ERROR; - } - - /** Proleptic Gregorian calendar date. YearT is an integral type - * which should be at least 32 bits wide, and should preferably - * be signed. - */ - template - class GregorianDate - { - public: - /** Construct from date in text form 'YYYY-MM-DD' by reading from - * ReadBuffer. - */ - explicit GregorianDate(ReadBuffer & in); - - /** Construct from Modified Julian Day. The type T is an - * integral type which should be at least 32 bits wide, and - * should preferably signed. - */ - explicit GregorianDate(is_integer auto modified_julian_day); - - /** Convert to Modified Julian Day. The type T is an integral type - * which should be at least 32 bits wide, and should preferably - * signed. - */ - template - T toModifiedJulianDay() const; - - /** Write the date in text form 'YYYY-MM-DD' to a buffer. - */ - void write(WriteBuffer & buf) const - { - writeImpl(buf); - } - - bool tryWrite(WriteBuffer & buf) const - { - return writeImpl(buf); - } - - /** Convert to a string in text form 'YYYY-MM-DD'. - */ - std::string toString() const; - - YearT year() const noexcept - { - return year_; - } - - uint8_t month() const noexcept - { - return month_; - } - - uint8_t dayOfMonth() const noexcept - { - return day_of_month_; - } - - private: - YearT year_ = 0; - uint8_t month_ = 0; - uint8_t day_of_month_ = 0; - - template - ReturnType writeImpl(WriteBuffer & buf) const; - }; - - /** ISO 8601 Ordinal Date. YearT is an integral type which should - * be at least 32 bits wide, and should preferably signed. - */ - template - class OrdinalDate - { - public: - OrdinalDate(YearT year, uint16_t day_of_year); - /** Construct from Modified Julian Day. The type T is an - * integral type which should be at least 32 bits wide, and - * should preferably signed. - */ - template - explicit OrdinalDate(DayT modified_julian_day); +class ReadBuffer; +class WriteBuffer; - /** Convert to Modified Julian Day. The type T is an integral - * type which should be at least 32 bits wide, and should - * preferably be signed. - */ - template - T toModifiedJulianDay() const noexcept; - - YearT year() const noexcept - { - return year_; - } - - uint16_t dayOfYear() const noexcept - { - return day_of_year_; - } - - private: - YearT year_ = 0; - uint16_t day_of_year_ = 0; - }; - - class MonthDay - { - public: - /** Construct from month and day. */ - MonthDay(uint8_t month, uint8_t day_of_month); - - /** Construct from day of year in Gregorian or Julian - * calendars to month and day. - */ - MonthDay(bool is_leap_year, uint16_t day_of_year); - - /** Convert month and day in Gregorian or Julian calendars to - * day of year. - */ - uint16_t dayOfYear(bool is_leap_year) const; - - uint8_t month() const noexcept - { - return month_; - } - - uint8_t dayOfMonth() const noexcept - { - return day_of_month_; - } - - private: - uint8_t month_ = 0; - uint8_t day_of_month_ = 0; - }; -} - - -namespace gd +/// Proleptic Gregorian calendar date. +class GregorianDate { - using namespace DB; +public: + /** Construct from date in text form 'YYYY-MM-DD' by reading from + * ReadBuffer. + */ + explicit GregorianDate(ReadBuffer & in); - template - static inline constexpr bool is_leap_year(YearT year) - { - return (year % 4 == 0) && ((year % 400 == 0) || (year % 100 != 0)); - } + /** Construct from Modified Julian Day. The type T is an + * integral type which should be at least 32 bits wide, and + * should preferably signed. + */ + explicit GregorianDate(int64_t modified_julian_day); - static inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) - { - switch (month) - { - case 1: return 31; - case 2: return is_leap_year ? 29 : 28; - case 3: return 31; - case 4: return 30; - case 5: return 31; - case 6: return 30; - case 7: return 31; - case 8: return 31; - case 9: return 30; - case 10: return 31; - case 11: return 30; - case 12: return 31; - default: - std::terminate(); - } - } + /** Convert to Modified Julian Day. The type T is an integral type + * which should be at least 32 bits wide, and should preferably + * signed. + */ + int64_t toModifiedJulianDay() const; - /** Integer division truncated toward negative infinity. + /** Write the date in text form 'YYYY-MM-DD' to a buffer. */ - template - static inline constexpr I div(I x, J y) + void write(WriteBuffer & buf) const { - const auto y_cast = static_cast(y); - if (x > 0 && y_cast < 0) - return ((x - 1) / y_cast) - 1; - else if (x < 0 && y_cast > 0) - return ((x + 1) / y_cast) - 1; - else - return x / y_cast; + writeImpl(buf); } - /** Integer modulus, satisfying div(x, y)*y + mod(x, y) == x. - */ - template - static inline constexpr I mod(I x, J y) + bool tryWrite(WriteBuffer & buf) const { - const auto y_cast = static_cast(y); - const auto r = x % y_cast; - if ((x > 0 && y_cast < 0) || (x < 0 && y_cast > 0)) - return r == 0 ? static_cast(0) : r + y_cast; - else - return r; + return writeImpl(buf); } - /** Like std::min(), but the type of operands may differ. + /** Convert to a string in text form 'YYYY-MM-DD'. */ - template - static inline constexpr I min(I x, J y) - { - const auto y_cast = static_cast(y); - return x < y_cast ? x : y_cast; - } + std::string toString() const; - static inline char readDigit(ReadBuffer & in) + int32_t year() const noexcept { - char c; - if (!in.read(c)) - throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected a digit at the end of stream"); - else if (c < '0' || c > '9') - throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot read input: expected a digit but got something else"); - else - return c - '0'; + return year_; } -} -namespace DB -{ - template - GregorianDate::GregorianDate(ReadBuffer & in) + uint8_t month() const noexcept { - year_ = gd::readDigit(in) * 1000 - + gd::readDigit(in) * 100 - + gd::readDigit(in) * 10 - + gd::readDigit(in); - - assertChar('-', in); - - month_ = gd::readDigit(in) * 10 - + gd::readDigit(in); - - assertChar('-', in); - - day_of_month_ = gd::readDigit(in) * 10 - + gd::readDigit(in); - - assertEOF(in); - - if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > gd::monthLength(gd::is_leap_year(year_), month_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date: {}", toString()); + return month_; } - template - GregorianDate::GregorianDate(is_integer auto modified_julian_day) + uint8_t dayOfMonth() const noexcept { - const OrdinalDate ord(modified_julian_day); - const MonthDay md(gd::is_leap_year(ord.year()), ord.dayOfYear()); - - year_ = ord.year(); - month_ = md.month(); - day_of_month_ = md.dayOfMonth(); + return day_of_month_; } - template - template - T GregorianDate::toModifiedJulianDay() const - { - const MonthDay md(month_, day_of_month_); - const auto day_of_year = md.dayOfYear(gd::is_leap_year(year_)); - const OrdinalDate ord(year_, day_of_year); - return ord.template toModifiedJulianDay(); - } +private: + int32_t year_ = 0; + uint8_t month_ = 0; + uint8_t day_of_month_ = 0; - template template - ReturnType GregorianDate::writeImpl(WriteBuffer & buf) const - { - if (year_ < 0 || year_ > 9999) - { - if constexpr (std::is_same_v) - throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME, - "Impossible to stringify: year too big or small: {}", DB::toString(year_)); - else - return false; - } - else - { - auto y = year_; - writeChar('0' + y / 1000, buf); y %= 1000; - writeChar('0' + y / 100, buf); y %= 100; - writeChar('0' + y / 10, buf); y %= 10; - writeChar('0' + y , buf); - - writeChar('-', buf); - - auto m = month_; - writeChar('0' + m / 10, buf); m %= 10; - writeChar('0' + m , buf); + ReturnType writeImpl(WriteBuffer & buf) const; +}; - writeChar('-', buf); +/** ISO 8601 Ordinal Date. + */ +class OrdinalDate +{ +public: + OrdinalDate(int32_t year, uint16_t day_of_year); - auto d = day_of_month_; - writeChar('0' + d / 10, buf); d %= 10; - writeChar('0' + d , buf); - } + /** Construct from Modified Julian Day. The type T is an + * integral type which should be at least 32 bits wide, and + * should preferably signed. + */ + explicit OrdinalDate(int64_t modified_julian_day); - return ReturnType(true); - } + /** Convert to Modified Julian Day. The type T is an integral + * type which should be at least 32 bits wide, and should + * preferably be signed. + */ + int64_t toModifiedJulianDay() const noexcept; - template - std::string GregorianDate::toString() const + int32_t year() const noexcept { - WriteBufferFromOwnString buf; - write(buf); - return buf.str(); + return year_; } - template - OrdinalDate::OrdinalDate(YearT year, uint16_t day_of_year) - : year_(year) - , day_of_year_(day_of_year) + uint16_t dayOfYear() const noexcept { - if (day_of_year < 1 || day_of_year > (gd::is_leap_year(year) ? 366 : 365)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid ordinal date: {}-{}", toString(year), toString(day_of_year)); - } + return day_of_year_; } - template - template - OrdinalDate::OrdinalDate(DayT modified_julian_day) - { - /// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively). - - if constexpr (is_signed_v && std::numeric_limits::lowest() < -678941) - if (modified_julian_day < -678941) - throw Exception( - ErrorCodes::CANNOT_FORMAT_DATETIME, - "Value cannot be represented as date because it's out of range"); +private: + int32_t year_ = 0; + uint16_t day_of_year_ = 0; +}; - if constexpr (std::numeric_limits::max() > 2973119) - if (modified_julian_day > 2973119) - throw Exception( - ErrorCodes::CANNOT_FORMAT_DATETIME, - "Value cannot be represented as date because it's out of range"); +class MonthDay +{ +public: + /** Construct from month and day. */ + MonthDay(uint8_t month, uint8_t day_of_month); - const auto a = modified_julian_day + 678575; - const auto quad_cent = gd::div(a, 146097); - const auto b = gd::mod(a, 146097); - const auto cent = gd::min(gd::div(b, 36524), 3); - const auto c = b - cent * 36524; - const auto quad = gd::div(c, 1461); - const auto d = gd::mod(c, 1461); - const auto y = gd::min(gd::div(d, 365), 3); + /** Construct from day of year in Gregorian or Julian + * calendars to month and day. + */ + MonthDay(bool is_leap_year, uint16_t day_of_year); - day_of_year_ = d - y * 365 + 1; - year_ = static_cast(quad_cent * 400 + cent * 100 + quad * 4 + y + 1); - } + /** Convert month and day in Gregorian or Julian calendars to + * day of year. + */ + uint16_t dayOfYear(bool is_leap_year) const; - template - template - T OrdinalDate::toModifiedJulianDay() const noexcept + uint8_t month() const noexcept { - const auto y = year_ - 1; - return day_of_year_ - + 365 * y - + gd::div(y, 4) - - gd::div(y, 100) - + gd::div(y, 400) - - 678576; + return month_; } - inline MonthDay::MonthDay(uint8_t month, uint8_t day_of_month) - : month_(month) - , day_of_month_(day_of_month) + uint8_t dayOfMonth() const noexcept { - if (month < 1 || month > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid month: {}", DB::toString(month)); - /* We can't validate day_of_month here, because we don't know if - * it's a leap year. */ + return day_of_month_; } - inline MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) - { - if (day_of_year < 1 || day_of_year > (is_leap_year ? 366 : 365)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year: {}{}", - (is_leap_year ? "leap, " : "non-leap, "), DB::toString(day_of_year)); +private: + uint8_t month_ = 0; + uint8_t day_of_month_ = 0; +}; - month_ = 1; - uint16_t d = day_of_year; - while (true) - { - const auto len = gd::monthLength(is_leap_year, month_); - if (d <= len) - break; - month_++; - d -= len; - } - day_of_month_ = d; - } - - inline uint16_t MonthDay::dayOfYear(bool is_leap_year) const - { - if (day_of_month_ < 1 || day_of_month_ > gd::monthLength(is_leap_year, month_)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month: {}{}-{}", - (is_leap_year ? "leap, " : "non-leap, "), DB::toString(month_), DB::toString(day_of_month_)); - } - const auto k = month_ <= 2 ? 0 : is_leap_year ? -1 :-2; - return (367 * month_ - 362) / 12 + k + day_of_month_; - } } diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index a7c2c04bf012..bad0696e5032 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -13,6 +13,7 @@ #include #include + namespace DB { @@ -56,14 +57,14 @@ namespace DB { if constexpr (nullOnErrors) { - const GregorianDate<> gd(vec_from[i]); + const GregorianDate gd(vec_from[i]); (*vec_null_map_to)[i] = gd.tryWrite(write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } else { - const GregorianDate<> gd(vec_from[i]); + const GregorianDate gd(vec_from[i]); gd.write(write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); diff --git a/src/Functions/toModifiedJulianDay.cpp b/src/Functions/toModifiedJulianDay.cpp index 0d854bcc1106..f800b279385d 100644 --- a/src/Functions/toModifiedJulianDay.cpp +++ b/src/Functions/toModifiedJulianDay.cpp @@ -80,8 +80,8 @@ namespace DB { try { - const GregorianDate<> date(read_buffer); - vec_to[i] = date.toModifiedJulianDay(); + const GregorianDate date(read_buffer); + vec_to[i] = static_cast(date.toModifiedJulianDay()); vec_null_map_to[i] = false; } catch (const Exception & e) @@ -97,8 +97,8 @@ namespace DB } else { - const GregorianDate<> date(read_buffer); - vec_to[i] = date.toModifiedJulianDay(); + const GregorianDate date(read_buffer); + vec_to[i] = static_cast(date.toModifiedJulianDay()); } } From 2a6b5e4ec6134e5c6451301ddcfa5d6acd949567 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 23:28:45 +0200 Subject: [PATCH 090/226] Fixed bad code --- src/Functions/GregorianDate.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index 0f8a95ff3e74..38ed3e2ddf8f 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -115,7 +115,7 @@ GregorianDate::GregorianDate(ReadBuffer & in) assertEOF(in); if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > gd::monthLength(gd::is_leap_year(year_), month_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date: {}", toString()); + throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date"); } GregorianDate::GregorianDate(int64_t modified_julian_day) @@ -143,7 +143,7 @@ ReturnType GregorianDate::writeImpl(WriteBuffer & buf) const { if constexpr (std::is_same_v) throw Exception(ErrorCodes::CANNOT_FORMAT_DATETIME, - "Impossible to stringify: year too big or small: {}", DB::toString(year_)); + "Impossible to stringify: year too big or small: {}", year_); else return false; } @@ -231,7 +231,7 @@ MonthDay::MonthDay(uint8_t month, uint8_t day_of_month) , day_of_month_(day_of_month) { if (month < 1 || month > 12) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid month: {}", DB::toString(month)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid month: {}", month); /* We can't validate day_of_month here, because we don't know if * it's a leap year. */ } @@ -240,7 +240,7 @@ MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) { if (day_of_year < 1 || day_of_year > (is_leap_year ? 366 : 365)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of year: {}{}", - (is_leap_year ? "leap, " : "non-leap, "), DB::toString(day_of_year)); + (is_leap_year ? "leap, " : "non-leap, "), day_of_year); month_ = 1; uint16_t d = day_of_year; @@ -249,7 +249,7 @@ MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) const auto len = gd::monthLength(is_leap_year, month_); if (d <= len) break; - month_++; + ++month_; d -= len; } day_of_month_ = d; @@ -260,7 +260,7 @@ uint16_t MonthDay::dayOfYear(bool is_leap_year) const if (day_of_month_ < 1 || day_of_month_ > gd::monthLength(is_leap_year, month_)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month: {}{}-{}", - (is_leap_year ? "leap, " : "non-leap, "), DB::toString(month_), DB::toString(day_of_month_)); + (is_leap_year ? "leap, " : "non-leap, "), month_, day_of_month_); } const auto k = month_ <= 2 ? 0 : is_leap_year ? -1 :-2; return (367 * month_ - 362) / 12 + k + day_of_month_; From 24b9c430f83b938329d228abd62ed44845fa63fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jul 2023 00:39:10 +0200 Subject: [PATCH 091/226] Continuation --- src/Functions/GregorianDate.cpp | 186 ++++++++++++++++++------ src/Functions/GregorianDate.h | 17 +++ src/Functions/fromModifiedJulianDay.cpp | 6 +- src/Functions/toModifiedJulianDay.cpp | 23 +-- 4 files changed, 172 insertions(+), 60 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index 38ed3e2ddf8f..da1172c8916a 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -19,7 +18,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace gd +namespace { static inline constexpr bool is_leap_year(int32_t year) { @@ -93,49 +92,129 @@ namespace gd else return c - '0'; } + + static inline bool tryReadDigit(ReadBuffer & in, char & c) + { + if (in.read(c) && c >= '0' && c <= '9') + { + c -= '0'; + return true; + } + + return false; + } } -GregorianDate::GregorianDate(ReadBuffer & in) +void GregorianDate::init(ReadBuffer & in) { - year_ = gd::readDigit(in) * 1000 - + gd::readDigit(in) * 100 - + gd::readDigit(in) * 10 - + gd::readDigit(in); + year_ = readDigit(in) * 1000 + + readDigit(in) * 100 + + readDigit(in) * 10 + + readDigit(in); assertChar('-', in); - month_ = gd::readDigit(in) * 10 - + gd::readDigit(in); + month_ = readDigit(in) * 10 + + readDigit(in); assertChar('-', in); - day_of_month_ = gd::readDigit(in) * 10 - + gd::readDigit(in); + day_of_month_ = readDigit(in) * 10 + + readDigit(in); assertEOF(in); - if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > gd::monthLength(gd::is_leap_year(year_), month_)) + if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year(year_), month_)) throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date"); } -GregorianDate::GregorianDate(int64_t modified_julian_day) +bool GregorianDate::tryInit(ReadBuffer & in) +{ + char c[8]; + + if ( !tryReadDigit(in, c[0]) + || !tryReadDigit(in, c[1]) + || !tryReadDigit(in, c[2]) + || !tryReadDigit(in, c[3]) + || !checkChar('-', in) + || !tryReadDigit(in, c[4]) + || !tryReadDigit(in, c[5]) + || !checkChar('-', in) + || !tryReadDigit(in, c[6]) + || !tryReadDigit(in, c[7]) + || !in.eof()) + { + return false; + } + + year_ = c[0] * 1000 + c[1] * 100 + c[2] * 10 + c[3]; + month_ = c[4] * 10 + c[5]; + day_of_month_ = c[6] * 10 + c[7]; + + if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year(year_), month_)) + return false; + + return true; +} + +GregorianDate::GregorianDate(ReadBuffer & in) +{ + init(in); +} + +void GregorianDate::init(int64_t modified_julian_day) { const OrdinalDate ord(modified_julian_day); - const MonthDay md(gd::is_leap_year(ord.year()), ord.dayOfYear()); + const MonthDay md(is_leap_year(ord.year()), ord.dayOfYear()); - year_ = ord.year(); - month_ = md.month(); + year_ = ord.year(); + month_ = md.month(); day_of_month_ = md.dayOfMonth(); } +bool GregorianDate::tryInit(int64_t modified_julian_day) +{ + OrdinalDate ord; + if (!ord.tryInit(modified_julian_day)) + return false; + + MonthDay md(is_leap_year(ord.year()), ord.dayOfYear()); + + year_ = ord.year(); + month_ = md.month(); + day_of_month_ = md.dayOfMonth(); + + return true; +} + +GregorianDate::GregorianDate(int64_t modified_julian_day) +{ + init(modified_julian_day); +} + int64_t GregorianDate::toModifiedJulianDay() const { const MonthDay md(month_, day_of_month_); - const auto day_of_year = md.dayOfYear(gd::is_leap_year(year_)); + + const auto day_of_year = md.dayOfYear(is_leap_year(year_)); + const OrdinalDate ord(year_, day_of_year); return ord.toModifiedJulianDay(); } +bool GregorianDate::tryToModifiedJulianDay(int64_t & res) const +{ + const MonthDay md(month_, day_of_month_); + const auto day_of_year = md.dayOfYear(is_leap_year(year_)); + OrdinalDate ord; + + if (!ord.tryInit(year_, day_of_year)) + return false; + + res = ord.toModifiedJulianDay(); + return true; +} + template ReturnType GregorianDate::writeImpl(WriteBuffer & buf) const { @@ -178,51 +257,76 @@ std::string GregorianDate::toString() const return buf.str(); } -OrdinalDate::OrdinalDate(int32_t year, uint16_t day_of_year) - : year_(year) - , day_of_year_(day_of_year) +void OrdinalDate::init(int32_t year, uint16_t day_of_year) { - if (day_of_year < 1 || day_of_year > (gd::is_leap_year(year) ? 366 : 365)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid ordinal date: {}-{}", toString(year), toString(day_of_year)); - } + year_ = year; + day_of_year_ = day_of_year; + + if (day_of_year < 1 || day_of_year > (is_leap_year(year) ? 366 : 365)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid ordinal date: {}-{}", year, day_of_year); } -OrdinalDate::OrdinalDate(int64_t modified_julian_day) +bool OrdinalDate::tryInit(int32_t year, uint16_t day_of_year) { - /// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively). + year_ = year; + day_of_year_ = day_of_year; - if (modified_julian_day < -678941) + return !(day_of_year < 1 || day_of_year > (is_leap_year(year) ? 366 : 365)); +} + +void OrdinalDate::init(int64_t modified_julian_day) +{ + if (!tryInit(modified_julian_day)) throw Exception( ErrorCodes::CANNOT_FORMAT_DATETIME, "Value cannot be represented as date because it's out of range"); +} + +bool OrdinalDate::tryInit(int64_t modified_julian_day) +{ + /// This function supports day number from -678941 to 2973119 (which represent 0000-01-01 and 9999-12-31 respectively). + + if (modified_julian_day < -678941) + return false; if (modified_julian_day > 2973119) - throw Exception( - ErrorCodes::CANNOT_FORMAT_DATETIME, - "Value cannot be represented as date because it's out of range"); + return false; const auto a = modified_julian_day + 678575; - const auto quad_cent = gd::div(a, 146097); - const auto b = gd::mod(a, 146097); - const auto cent = gd::min(gd::div(b, 36524), 3); + const auto quad_cent = div(a, 146097); + const auto b = mod(a, 146097); + const auto cent = min(div(b, 36524), 3); const auto c = b - cent * 36524; - const auto quad = gd::div(c, 1461); - const auto d = gd::mod(c, 1461); - const auto y = gd::min(gd::div(d, 365), 3); + const auto quad = div(c, 1461); + const auto d = mod(c, 1461); + const auto y = min(div(d, 365), 3); day_of_year_ = d - y * 365 + 1; year_ = static_cast(quad_cent * 400 + cent * 100 + quad * 4 + y + 1); + + return true; +} + + +OrdinalDate::OrdinalDate(int32_t year, uint16_t day_of_year) +{ + init(year, day_of_year); +} + +OrdinalDate::OrdinalDate(int64_t modified_julian_day) +{ + init(modified_julian_day); } int64_t OrdinalDate::toModifiedJulianDay() const noexcept { const auto y = year_ - 1; + return day_of_year_ + 365 * y - + gd::div(y, 4) - - gd::div(y, 100) - + gd::div(y, 400) + + div(y, 4) + - div(y, 100) + + div(y, 400) - 678576; } @@ -246,7 +350,7 @@ MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) uint16_t d = day_of_year; while (true) { - const auto len = gd::monthLength(is_leap_year, month_); + const auto len = monthLength(is_leap_year, month_); if (d <= len) break; ++month_; @@ -257,7 +361,7 @@ MonthDay::MonthDay(bool is_leap_year, uint16_t day_of_year) uint16_t MonthDay::dayOfYear(bool is_leap_year) const { - if (day_of_month_ < 1 || day_of_month_ > gd::monthLength(is_leap_year, month_)) + if (day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year, month_)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid day of month: {}{}-{}", (is_leap_year ? "leap, " : "non-leap, "), month_, day_of_month_); diff --git a/src/Functions/GregorianDate.h b/src/Functions/GregorianDate.h index 4a0cbec5afe6..2528223443e6 100644 --- a/src/Functions/GregorianDate.h +++ b/src/Functions/GregorianDate.h @@ -13,11 +13,19 @@ class WriteBuffer; class GregorianDate { public: + GregorianDate() {} + + void init(ReadBuffer & in); + bool tryInit(ReadBuffer & in); + /** Construct from date in text form 'YYYY-MM-DD' by reading from * ReadBuffer. */ explicit GregorianDate(ReadBuffer & in); + void init(int64_t modified_julian_day); + bool tryInit(int64_t modified_julian_day); + /** Construct from Modified Julian Day. The type T is an * integral type which should be at least 32 bits wide, and * should preferably signed. @@ -29,6 +37,7 @@ class GregorianDate * signed. */ int64_t toModifiedJulianDay() const; + bool tryToModifiedJulianDay(int64_t & res) const; /** Write the date in text form 'YYYY-MM-DD' to a buffer. */ @@ -75,6 +84,14 @@ class GregorianDate class OrdinalDate { public: + OrdinalDate() {} + + void init(int32_t year, uint16_t day_of_year); + bool tryInit(int32_t year, uint16_t day_of_year); + + void init(int64_t modified_julian_day); + bool tryInit(int64_t modified_julian_day); + OrdinalDate(int32_t year, uint16_t day_of_year); /** Construct from Modified Julian Day. The type T is an diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index bad0696e5032..8736b1fce7fe 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -57,14 +57,14 @@ namespace DB { if constexpr (nullOnErrors) { - const GregorianDate gd(vec_from[i]); - (*vec_null_map_to)[i] = gd.tryWrite(write_buffer); + GregorianDate gd; + (*vec_null_map_to)[i] = !(gd.tryInit(vec_from[i]) && gd.tryWrite(write_buffer)); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } else { - const GregorianDate gd(vec_from[i]); + GregorianDate gd(vec_from[i]); gd.write(write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); diff --git a/src/Functions/toModifiedJulianDay.cpp b/src/Functions/toModifiedJulianDay.cpp index f800b279385d..5b4cd34141c2 100644 --- a/src/Functions/toModifiedJulianDay.cpp +++ b/src/Functions/toModifiedJulianDay.cpp @@ -78,22 +78,13 @@ namespace DB if constexpr (nullOnErrors) { - try - { - const GregorianDate date(read_buffer); - vec_to[i] = static_cast(date.toModifiedJulianDay()); - vec_null_map_to[i] = false; - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || e.code() == ErrorCodes::CANNOT_PARSE_DATE) - { - vec_to[i] = static_cast(0); - vec_null_map_to[i] = true; - } - else - throw; - } + GregorianDate date; + + int64_t res = 0; + bool success = date.tryInit(read_buffer) && date.tryToModifiedJulianDay(res); + + vec_to[i] = static_cast(res); + vec_null_map_to[i] = !success; } else { From d8a66a81233441676fdd8f0c786060c2b1aacd56 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jul 2023 17:49:01 +0800 Subject: [PATCH 092/226] fix asan error --- src/Functions/substringIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index d1791c9696b7..5f3f054b6240 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -208,7 +208,7 @@ namespace { size_t res_offset = res_data.size(); res_data.resize(res_offset + res_ref.size + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size); + memcpy(&res_data[res_offset], res_ref.data, res_ref.size); res_offset += res_ref.size; res_data[res_offset] = 0; ++res_offset; From baee73fd96d1b1974ac7ec637c3b22c4f63a27a4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:11:25 +0200 Subject: [PATCH 093/226] Make shutdown of replicated tables softer --- programs/server/Server.cpp | 64 ++++++--- programs/server/Server.h | 11 +- src/Databases/DatabasesCommon.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Storages/IStorage.h | 8 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 123 +++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 11 +- src/Storages/StorageTableFunction.h | 4 +- .../__init__.py | 1 + .../config/merge_tree_conf.xml | 5 + .../test.py | 74 +++++++++++ 19 files changed, 246 insertions(+), 74 deletions(-) create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d2d8a0d07fb7..0a311fa4737f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1286,7 +1286,7 @@ try global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } global_context->updateStorageConfiguration(*config); @@ -1388,10 +1388,15 @@ try } - for (auto & server : servers_to_start_before_tables) { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); + std::lock_guard lock(servers_lock); + createInterserverServers(config(), interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); + + for (auto & server : servers_to_start_before_tables) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } } /// Initialize access storages. @@ -1688,7 +1693,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " @@ -1954,7 +1959,6 @@ HTTPContextPtr Server::httpContext() const void Server::createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, - const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, @@ -2176,6 +2180,23 @@ void Server::createServers( httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } +} + +void Server::createInterserverServers( + Poco::Util::AbstractConfiguration & config, + const Strings & interserver_listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); /// Now iterate over interserver_listen_hosts for (const auto & interserver_listen_host : interserver_listen_hosts) @@ -2224,14 +2245,14 @@ void Server::createServers( #endif }); } - } void Server::updateServers( Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers) + std::vector & servers, + std::vector & servers_to_start_before_tables) { Poco::Logger * log = &logger(); @@ -2256,12 +2277,18 @@ void Server::updateServers( std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); - + std::vector all_servers; for (auto & server : servers) + all_servers.push_back(&server); + + for (auto & server : servers_to_start_before_tables) + all_servers.push_back(&server); + + for (auto * server : all_servers) { - if (!server.isStopping()) + if (!server->isStopping()) { - std::string port_name = server.getPortName(); + std::string port_name = server->getPortName(); bool has_host = false; bool is_http = false; if (port_name.starts_with("protocols.")) @@ -2299,25 +2326,26 @@ void Server::updateServers( /// NOTE: better to compare using getPortName() over using /// dynamic_cast<> since HTTPServer is also used for prometheus and /// internal replication communications. - is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port"; + is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port"; } if (!has_host) - has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); + has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end(); bool has_port = !config.getString(port_name, "").empty(); bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); if (force_restart) - LOG_TRACE(log, " had been changed, will reload {}", server.getDescription()); + LOG_TRACE(log, " had been changed, will reload {}", server->getDescription()); - if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart) + if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart) { - server.stop(); - LOG_INFO(log, "Stopped listening for {}", server.getDescription()); + server->stop(); + LOG_INFO(log, "Stopped listening for {}", server->getDescription()); } } } - createServers(config, listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); std::erase_if(servers, std::bind_front(check_server, "")); } diff --git a/programs/server/Server.h b/programs/server/Server.h index e9ae6d8d937d..d13378dcd653 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -102,6 +102,14 @@ class Server : public BaseDaemon, public IServer void createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false); + + void createInterserverServers( + Poco::Util::AbstractConfiguration & config, const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, @@ -113,7 +121,8 @@ class Server : public BaseDaemon, public IServer Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers); + std::vector & servers, + std::vector & servers_to_start_before_tables); }; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index bb98e2bd3bbd..4ba793d858dc 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -292,7 +292,7 @@ void DatabaseWithOwnTablesBase::shutdown() for (const auto & kv : tables_snapshot) { - kv.second->flush(); + kv.second->flushAndPrepareForShutdown(); } for (const auto & kv : tables_snapshot) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 0beb4492aef3..84ecb1fc9094 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -361,7 +361,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, std::vector> tables_to_drop; for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { - iterator->table()->flush(); + iterator->table()->flushAndPrepareForShutdown(); tables_to_drop.push_back({iterator->name(), iterator->table()->isDictionary()}); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b262d88db571..c0d368364445 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -549,15 +549,15 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /** * If the storage requires some complicated work on destroying, * then you have two virtual methods: - * - flush() + * - flushAndPrepareForShutdown() * - shutdown() * * @see shutdown() - * @see flush() + * @see flushAndPrepareForShutdown() */ void flushAndShutdown() { - flush(); + flushAndPrepareForShutdown(); shutdown(); } @@ -570,7 +570,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// Called before shutdown() to flush data to underlying storage /// Data in memory need to be persistent - virtual void flush() {} + virtual void flushAndPrepareForShutdown() {} /// Asks table to stop executing some action identified by action_type /// If table does not support such type of lock, and empty lock is returned diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7424a2484917..fc8f599a06e3 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -204,7 +204,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write else sendPartFromDisk(part, out, client_protocol_version, false, send_projections); - data.addLastSentPart(part->name); + data.addLastSentPart(part->info); } catch (const NetException &) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dc24327712cf..60c3999f87a4 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -119,6 +119,7 @@ struct Settings; M(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ + M(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index d7166b4a3b93..0e381654db05 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -330,7 +330,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown) { setReadonly(part_of_full_shutdown); - storage.partialShutdown(); + storage.partialShutdown(part_of_full_shutdown); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index d021667f771d..9c05afd52849 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -682,7 +682,7 @@ void StorageBuffer::startup() } -void StorageBuffer::flush() +void StorageBuffer::flushAndPrepareForShutdown() { if (!flush_handle) return; diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 8f089a4d580e..db3cde93be5d 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -92,7 +92,7 @@ friend class BufferSink; void startup() override; /// Flush all buffers into the subordinate table and stop background thread. - void flush() override; + void flushAndPrepareForShutdown() override; bool optimize( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b9..608f65cfeff7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1427,7 +1427,7 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) return {}; } -void StorageDistributed::flush() +void StorageDistributed::flushAndPrepareForShutdown() { try { diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f45286341cfc..547f61a012b1 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -135,7 +135,7 @@ class StorageDistributed final : public IStorage, WithContext void initializeFromDisk(); void shutdown() override; - void flush() override; + void flushAndPrepareForShutdown() override; void drop() override; bool storesDataOnDisk() const override { return data_volume != nullptr; } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 14b7fc15af27..b243225adb3b 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -139,7 +139,7 @@ class StorageProxy : public IStorage void startup() override { getNested()->startup(); } void shutdown() override { getNested()->shutdown(); } - void flush() override { getNested()->flush(); } + void flushAndPrepareForShutdown() override { getNested()->flushAndPrepareForShutdown(); } ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7f282b6c0e67..94727a5495ce 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -3933,6 +3934,7 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) { std::lock_guard lock(last_sent_parts_mutex); last_sent_parts.emplace_back(info); + static constexpr size_t LAST_SENT_PARS_WINDOW_SIZE = 1000; while (last_sent_parts.size() > LAST_SENT_PARS_WINDOW_SIZE) last_sent_parts.pop_front(); } @@ -3950,24 +3952,32 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si auto zookeeper = getZooKeeper(); - auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper); + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); if (unique_parts_set.empty()) { LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); return; } + else + { + LOG_INFO(log, "Will wait for {} unique parts to be fetched", unique_parts_set.size()); + } - auto wait_predicate = [&] () -> void + auto wait_predicate = [&] () -> bool { bool all_fetched = true; - for (const auto & part : unique_parts_set) + for (auto it = unique_parts_set.begin(); it != unique_parts_set.end();) { + const auto & part = *it; + bool found = false; - for (const auto & sent_part : last_sent_parts) + for (const auto & sent_part : last_sent_parts | std::views::reverse) { if (sent_part.contains(part)) { + LOG_TRACE(log, "Part {} was fetched by some replica", part.getPartNameForLogs()); found = true; + it = unique_parts_set.erase(it); break; } } @@ -3981,14 +3991,19 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si }; std::unique_lock lock(last_sent_parts_mutex); - if (!last_sent_parts_cv.wait_for(last_sent_parts_cv, std::chrono::duration_cast(wait_ms), wait_predicate)) - LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unqiue_parts_set.size()); + if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) + LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + else + LOG_INFO(log, "Successfuly waited all the parts"); } -std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_) +std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) { - if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) + if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) + { + LOG_INFO(log_, "Our replica is not active, nobody will try to fetch anything"); return {}; + } Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); Strings our_parts; @@ -3996,40 +4011,54 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart for (const String & replica : replicas) { if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + { + LOG_TRACE(log_, "Replica {} is not active, skipping", replica); continue; + } Strings parts = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas" / replica / "parts"); if (replica == replica_name_) { + LOG_TRACE(log_, "Our replica parts collected {}", replica); our_parts = parts; } else { + LOG_TRACE(log_, "Fetching parts for replica {}", replica); data_parts_on_replicas.emplace_back(format_version_); for (const auto & part : parts) { - if (!data_parts_on_replicas.back().getContainingPart(part).empty()) + if (data_parts_on_replicas.back().getContainingPart(part).empty()) data_parts_on_replicas.back().add(part); } } } - NameSet our_unique_parts; + std::vector our_unique_parts; for (const auto & part : our_parts) { + LOG_TRACE(log_, "Looking for part {}", part); bool found = false; for (const auto & active_parts_set : data_parts_on_replicas) { if (!active_parts_set.getContainingPart(part).empty()) { + LOG_TRACE(log_, "Part {} found", part); found = true; break; } } + if (!found) - our_unique_parts.insert(MergeTreePartInfo::fromPartName(part, format_version)); + { + LOG_TRACE(log_, "Part not {} found", part); + our_unique_parts.emplace_back(MergeTreePartInfo::fromPartName(part, format_version_)); + } } + if (!our_parts.empty() && our_unique_parts.empty()) + LOG_TRACE(log_, "All parts found on replica"); + return our_unique_parts; } @@ -4799,14 +4828,53 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } -void StorageReplicatedMergeTree::partialShutdown() +void StorageReplicatedMergeTree::flushAndPrepareForShutdown() +{ + if (shutdown_prepared_called.exchange(true)) + return; + + session_expired_callback_handler.reset(); + stopOutdatedDataPartsLoadingTask(); + + /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. + fetcher.blocker.cancelForever(); + merger_mutator.merges_blocker.cancelForever(); + parts_mover.moves_blocker.cancelForever(); + mutations_finalizing_task->deactivate(); + stopBeingLeader(); + + if (attach_thread) + attach_thread->shutdown(); + + restarting_thread.shutdown(/* part_of_full_shutdown */true); + background_operations_assignee.finish(); + part_moves_between_shards_orchestrator.shutdown(); + + { + auto lock = queue.lockQueue(); + /// Cancel logs pulling after background task were cancelled. It's still + /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, + /// MUTATE, etc. query. + queue.pull_log_blocker.cancelForever(); + } + background_moves_assignee.finish(); + +} + +void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) { ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); partial_shutdown_called = true; partial_shutdown_event.set(); queue.notifySubscribersOnPartialShutdown(); - replica_is_active_node = nullptr; + if (!part_of_full_shutdown) + { + LOG_DEBUG(log, "Reset active node, replica will be inactive"); + replica_is_active_node = nullptr; + } + else + LOG_DEBUG(log, "Will not reset active node, it will be reset completely during full shutdown"); LOG_TRACE(log, "Waiting for threads to finish"); merge_selecting_task->deactivate(); @@ -4834,31 +4902,14 @@ void StorageReplicatedMergeTree::shutdown() if (shutdown_called.exchange(true)) return; - session_expired_callback_handler.reset(); - stopOutdatedDataPartsLoadingTask(); - - /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. - fetcher.blocker.cancelForever(); - merger_mutator.merges_blocker.cancelForever(); - parts_mover.moves_blocker.cancelForever(); - mutations_finalizing_task->deactivate(); - stopBeingLeader(); - - if (attach_thread) - attach_thread->shutdown(); + if (!shutdown_prepared_called.load()) + flushAndPrepareForShutdown(); - restarting_thread.shutdown(/* part_of_full_shutdown */true); - background_operations_assignee.finish(); - part_moves_between_shards_orchestrator.shutdown(); + auto settings_ptr = getSettings(); + LOG_DEBUG(log, "Data parts exchange still exists {}", data_parts_exchange_endpoint != nullptr); + waitForUniquePartsToBeFetchedByOtherReplicas(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds()); - { - auto lock = queue.lockQueue(); - /// Cancel logs pulling after background task were cancelled. It's still - /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, - /// MUTATE, etc. query. - queue.pull_log_blocker.cancelForever(); - } - background_moves_assignee.finish(); + replica_is_active_node = nullptr; auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4661f0a56daa..104062def4b6 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -113,7 +113,10 @@ class StorageReplicatedMergeTree final : public MergeTreeData void startup() override; void shutdown() override; - void partialShutdown(); + + void flushAndPrepareForShutdown() override; + + void partialShutdown(bool part_of_full_shutdown); ~StorageReplicatedMergeTree() override; static String getDefaultZooKeeperPath(const Poco::Util::AbstractConfiguration & config); @@ -453,9 +456,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET std::atomic shutdown_called {false}; + std::atomic shutdown_prepared_called {false}; - static constexpr size_t LAST_SENT_PARS_WINDOW_SIZE = 1000; - std::mutex last_sent_parts_mutex; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; std::deque last_sent_parts; @@ -711,7 +714,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData */ String findReplicaHavingCoveringPart(LogEntry & entry, bool active); String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name); - static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_); + static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 26cbe1f02334..5df050d1d0d6 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -79,11 +79,11 @@ class StorageTableFunctionProxy final : public StorageProxy nested->shutdown(); } - void flush() override + void flushAndPrepareForShutdown() override { std::lock_guard lock{nested_mutex}; if (nested) - nested->flush(); + nested->flushAndPrepareForShutdown(); } void drop() override diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py new file mode 100644 index 000000000000..e5a0d9b4834e --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml new file mode 100644 index 000000000000..8ff3bdf9a2fd --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml @@ -0,0 +1,5 @@ + + + 30000 + + diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py new file mode 100644 index 000000000000..75f0921646ed --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -0,0 +1,74 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from multiprocessing.dummy import Pool +import time + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True +) + +node2 = cluster.add_instance( + "node2", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_shutdown_and_wait(start_cluster): + + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()") + + node1.query("INSERT INTO test_table VALUES (0)") + node2.query("SYSTEM SYNC REPLICA test_table") + + assert node1.query("SELECT * FROM test_table") == "0\n" + assert node2.query("SELECT * FROM test_table") == "0\n" + + def soft_shutdown(node): + node.stop_clickhouse(kill=False, stop_wait_sec=60) + + p = Pool(50) + pm = PartitionManager() + + pm.partition_instances(node1, node2) + + def insert(value): + node1.query(f"INSERT INTO test_table VALUES ({value})") + + p.map(insert, range(1, 50)) + + # Start shutdown async + waiter = p.apply_async(soft_shutdown, (node1,)) + # to be sure that shutdown started + time.sleep(5) + + # node 2 partitioned and don't see any data + assert node2.query("SELECT * FROM test_table") == "0\n" + + # Restore network + pm.heal_all() + # wait for shutdown to finish + waiter.get() + + node2.query("SYSTEM SYNC REPLICA test_table", timeout=5) + + # check second replica has all data + assert node2.query("SELECT sum(value) FROM test_table") == "1225\n" + # and nothing in queue + assert node2.query("SELECT count() FROM system.replication_queue") == "0\n" + + # It can happend that the second replica is superfast + assert node1.contains_in_log("Successfuly waited all the parts") or node1.contains_in_log("All parts found on replica") From 085576efc9c256d996dacf0f43185146c46eb194 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 5 Jul 2023 16:22:58 +0000 Subject: [PATCH 094/226] Automatic style fix --- .../test.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 75f0921646ed..e3a2e7a0271c 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -9,13 +9,20 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True + "node1", + main_configs=["config/merge_tree_conf.xml"], + with_zookeeper=True, + stay_alive=True, ) node2 = cluster.add_instance( - "node2", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True + "node2", + main_configs=["config/merge_tree_conf.xml"], + with_zookeeper=True, + stay_alive=True, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -27,9 +34,10 @@ def start_cluster(): def test_shutdown_and_wait(start_cluster): - for i, node in enumerate([node1, node2]): - node.query(f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()") + node.query( + f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()" + ) node1.query("INSERT INTO test_table VALUES (0)") node2.query("SYSTEM SYNC REPLICA test_table") @@ -71,4 +79,6 @@ def insert(value): assert node2.query("SELECT count() FROM system.replication_queue") == "0\n" # It can happend that the second replica is superfast - assert node1.contains_in_log("Successfuly waited all the parts") or node1.contains_in_log("All parts found on replica") + assert node1.contains_in_log( + "Successfuly waited all the parts" + ) or node1.contains_in_log("All parts found on replica") From 88d3e1723a8a53270c0da62e581217442383cc5c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:31:47 +0200 Subject: [PATCH 095/226] Fixes and comments --- programs/server/Server.cpp | 16 +++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++ src/Storages/StorageReplicatedMergeTree.h | 28 +++++++++++++++++++-- 3 files changed, 43 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0a311fa4737f..58cf3e5d2101 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1390,6 +1390,10 @@ try { std::lock_guard lock(servers_lock); + /// We should start interserver communications before (and more imporant shutdown after) tables. + /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. + /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can + /// communicate with zookeeper, execute merges, etc. createInterserverServers(config(), interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); for (auto & server : servers_to_start_before_tables) @@ -1516,10 +1520,13 @@ try { LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); size_t current_connections = 0; - for (auto & server : servers_to_start_before_tables) { - server.stop(); - current_connections += server.currentConnections(); + std::lock_guard lock(servers_lock); + for (auto & server : servers_to_start_before_tables) + { + server.stop(); + current_connections += server.currentConnections(); + } } if (current_connections) @@ -2345,9 +2352,10 @@ void Server::updateServers( } createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); - createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true); std::erase_if(servers, std::bind_front(check_server, "")); + std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, "")); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 94727a5495ce..2f165a056a35 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4870,6 +4870,11 @@ void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) queue.notifySubscribersOnPartialShutdown(); if (!part_of_full_shutdown) { + /// If we are going to completely shutdown table we allow other + /// replicas to fetch parts which are unique for our replica. + /// + /// Replicas try to fetch part only in case the source replica is active, + /// so don't reset handler here. LOG_DEBUG(log, "Reset active node, replica will be inactive"); replica_is_active_node = nullptr; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 104062def4b6..baa5af824b4c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -112,11 +112,35 @@ class StorageReplicatedMergeTree final : public MergeTreeData bool need_check_structure); void startup() override; - void shutdown() override; + /// To many shutdown methods.... + /// + /// Partial shutdown called if we loose connection to zookeeper. + /// Table can also recover after partial shutdown and continue + /// to work. This method can be called regularly. + void partialShutdown(bool part_of_full_shutdown); + + /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). + /// The shutdown process is splitted into two methods to make it more soft and fast. In database shutdown() + /// looks like: + /// for (table : tables) + /// table->flushAndPrepareForShutdown() + /// + /// for (table : tables) + /// table->shutdown() + /// + /// So we stop producting all the parts first for all tables (fast operation). And after we can wait in shutdown() + /// for other replicas to download parts. + /// + /// In flushAndPrepareForShutdown we cancel all part-producing operations: + /// merges, fetches, moves and so on. If it wasn't called before shutdown() -- shutdown() will + /// call it (defensive programming). void flushAndPrepareForShutdown() override; + /// In shutdown we completly terminate table -- remove + /// is_active node and interserver handler. Also optionally + /// wait until other replicas will download some parts from our replica. + void shutdown() override; - void partialShutdown(bool part_of_full_shutdown); ~StorageReplicatedMergeTree() override; static String getDefaultZooKeeperPath(const Poco::Util::AbstractConfiguration & config); From 2fa45117edfa4b541ad96f056a58e040c0569e4f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:38:04 +0200 Subject: [PATCH 096/226] Beter --- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++- src/Storages/StorageReplicatedMergeTree.h | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2f165a056a35..e5abf63a72d8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3944,13 +3944,16 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms) { + if (!shutdown_called.load()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); + if (wait_ms == 0) { LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because wait time is zero"); return; } - auto zookeeper = getZooKeeper(); + auto zookeeper = getZooKeeperIfTableShutDown(); auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); if (unique_parts_set.empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index baa5af824b4c..a1a0717ca64d 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -374,6 +374,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData return last_sent_parts; } + /// Wait required amount of milliseconds to give other replicas a chance to + /// download unique parts from our replica void waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms); private: From f2d106ffb783b9410cecdfe0e332d660b9fa73fa Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:54:33 +0200 Subject: [PATCH 097/226] Fix typos --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.h | 6 +++--- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e5abf63a72d8..114465df4968 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3995,9 +3995,9 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si std::unique_lock lock(last_sent_parts_mutex); if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) - LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else - LOG_INFO(log, "Successfuly waited all the parts"); + LOG_INFO(log, "Successfully waited all the parts"); } std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a1a0717ca64d..656e8df6ccb0 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -121,7 +121,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void partialShutdown(bool part_of_full_shutdown); /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). - /// The shutdown process is splitted into two methods to make it more soft and fast. In database shutdown() + /// The shutdown process is split into two methods to make it more soft and fast. In database shutdown() /// looks like: /// for (table : tables) /// table->flushAndPrepareForShutdown() @@ -129,14 +129,14 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// for (table : tables) /// table->shutdown() /// - /// So we stop producting all the parts first for all tables (fast operation). And after we can wait in shutdown() + /// So we stop producing all the parts first for all tables (fast operation). And after we can wait in shutdown() /// for other replicas to download parts. /// /// In flushAndPrepareForShutdown we cancel all part-producing operations: /// merges, fetches, moves and so on. If it wasn't called before shutdown() -- shutdown() will /// call it (defensive programming). void flushAndPrepareForShutdown() override; - /// In shutdown we completly terminate table -- remove + /// In shutdown we completely terminate table -- remove /// is_active node and interserver handler. Also optionally /// wait until other replicas will download some parts from our replica. void shutdown() override; diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index e3a2e7a0271c..a2a4ec92cf77 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -80,5 +80,5 @@ def insert(value): # It can happend that the second replica is superfast assert node1.contains_in_log( - "Successfuly waited all the parts" + "Successfully waited all the parts" ) or node1.contains_in_log("All parts found on replica") From 45db928e4e31aae6a6d7e8e6b35e0a5a3768375c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 02:52:55 +0200 Subject: [PATCH 098/226] Fix style --- src/Functions/fromModifiedJulianDay.cpp | 1 - src/Functions/toModifiedJulianDay.cpp | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 8736b1fce7fe..695d1b7d63c1 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_FORMAT_DATETIME; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } diff --git a/src/Functions/toModifiedJulianDay.cpp b/src/Functions/toModifiedJulianDay.cpp index 5b4cd34141c2..907c7570ce2d 100644 --- a/src/Functions/toModifiedJulianDay.cpp +++ b/src/Functions/toModifiedJulianDay.cpp @@ -17,8 +17,6 @@ namespace DB { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_PARSE_DATE; } template From 810d1ee0694cc769170f4b08c58aa4c2c5b0807a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Jul 2023 13:48:57 +0000 Subject: [PATCH 099/226] Fix tests --- src/Processors/Formats/IRowInputFormat.h | 2 +- .../Formats/Impl/ArrowBlockInputFormat.h | 2 +- .../Impl/JSONColumnsBlockInputFormatBase.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 2 +- .../Formats/Impl/ORCBlockInputFormat.h | 2 +- .../Formats/Impl/ParallelParsingInputFormat.h | 2 +- .../Formats/Impl/ParquetBlockInputFormat.h | 2 +- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 22 +++++++++++++------ 9 files changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index b7b1b0b29a67..00888cfa5e9c 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -85,7 +85,7 @@ class IRowInputFormat : public IInputFormat size_t num_errors = 0; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index df77994c3d58..2db8bd6c59c9 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -50,7 +50,7 @@ class ArrowBlockInputFormat : public IInputFormat int record_batch_current = 0; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index 5ab20c796ea3..bb52e2aa516c 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -67,7 +67,7 @@ class JSONColumnsBlockInputFormatBase : public IInputFormat Serializations serializations; std::unique_ptr reader; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index f8c9a39eedf1..65ea87479a34 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -66,7 +66,7 @@ class NativeInputFormat final : public IInputFormat std::unique_ptr reader; Block header; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; class NativeOutputFormat final : public IOutputFormat diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 98561e72e611..7097ea3ac080 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -52,7 +52,7 @@ class ORCBlockInputFormat : public IInputFormat std::vector include_indices; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; const FormatSettings format_settings; const std::unordered_set & skip_stripes; diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 4495680f5b29..f61dc3fbc780 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -202,7 +202,7 @@ class ParallelParsingInputFormat : public IInputFormat const size_t max_block_size; BlockMissingValues last_block_missing_values; - size_t last_approx_bytes_read_for_chunk; + size_t last_approx_bytes_read_for_chunk = 0; /// Non-atomic because it is used in one thread. std::optional next_block_in_current_unit; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a14c51f8b9f6..dc14edf2099d 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -273,7 +273,7 @@ class ParquetBlockInputFormat : public IInputFormat std::unique_ptr pool; BlockMissingValues previous_block_missing_values; - size_t previous_approx_bytes_read_for_chunk; + size_t previous_approx_bytes_read_for_chunk = 0; std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index d540a24fa706..8f8d44ec0884 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -96,7 +96,7 @@ class ValuesBlockInputFormat final : public IInputFormat Serializations serializations; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; class ValuesSchemaReader : public IRowSchemaReader diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index b9be01cf2aed..e583d2e30b7c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -301,18 +301,26 @@ class HDFSSource::URISIterator::Impl : WithContext StorageHDFS::PathWithInfo next() { - size_t current_index = index.fetch_add(1); - if (current_index >= uris.size()) - return {"", {}}; + String uri; + hdfsFileInfo * hdfs_info; + do + { + size_t current_index = index.fetch_add(1); + if (current_index >= uris.size()) + return {"", {}}; + + uri = uris[current_index]; + auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); + hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); + } + /// Skip non-existed files. + while (String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); - auto uri = uris[current_index]; - auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - auto * hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); std::optional info; if (hdfs_info) { info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - if (file_progress_callback && hdfs_info) + if (file_progress_callback) file_progress_callback(FileProgress(0, hdfs_info->mSize)); } From 9ae0dc730c586a37f8fdbbd880267ec11c2c8e51 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:15:26 +0200 Subject: [PATCH 100/226] Review fixes + enable in ci --- docker/test/upgrade/run.sh | 2 + .../ReplicatedMergeTreeRestartingThread.cpp | 7 +- .../ReplicatedMergeTreeRestartingThread.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 99 +++++++++---------- src/Storages/StorageReplicatedMergeTree.h | 14 ++- ...le_wait_for_shutdown_replicated_tables.xml | 5 + tests/config/install.sh | 1 + 7 files changed, 66 insertions(+), 63 deletions(-) create mode 100644 tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 82a88272df9b..07e6e7dd0ecb 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -61,6 +61,7 @@ configure # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start @@ -90,6 +91,7 @@ configure # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 0e381654db05..e43cc879e93e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -329,8 +329,8 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown) { - setReadonly(part_of_full_shutdown); - storage.partialShutdown(part_of_full_shutdown); + setReadonly(/* on_shutdown = */ part_of_full_shutdown); + storage.partialShutdown(); } @@ -341,8 +341,7 @@ void ReplicatedMergeTreeRestartingThread::shutdown(bool part_of_full_shutdown) task->deactivate(); LOG_TRACE(log, "Restarting thread finished"); - /// Stop other tasks. - partialShutdown(part_of_full_shutdown); + setReadonly(part_of_full_shutdown); } void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 9e99baab4c34..01a877a07e5e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -36,6 +36,7 @@ class ReplicatedMergeTreeRestartingThread void shutdown(bool part_of_full_shutdown); void run(); + private: StorageReplicatedMergeTree & storage; String log_name; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 114465df4968..88bd788b9efb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3942,17 +3942,26 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) last_sent_parts_cv.notify_all(); } -void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms) +void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(StorageReplicatedMergeTree::ShutdownDeadline shutdown_deadline_) { if (!shutdown_called.load()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); + auto settings_ptr = getSettings(); + + auto wait_ms = settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds(); if (wait_ms == 0) { LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because wait time is zero"); return; } + if (shutdown_deadline_ <= std::chrono::system_clock::now()) + { + LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because shutdown_deadline already passed"); + return; + } + auto zookeeper = getZooKeeperIfTableShutDown(); auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); @@ -3968,7 +3977,6 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si auto wait_predicate = [&] () -> bool { - bool all_fetched = true; for (auto it = unique_parts_set.begin(); it != unique_parts_set.end();) { const auto & part = *it; @@ -3985,22 +3993,19 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si } } if (!found) - { - all_fetched = false; break; - } } - return all_fetched; + return unique_parts_set.empty(); }; std::unique_lock lock(last_sent_parts_mutex); - if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) + if (!last_sent_parts_cv.wait_until(lock, shutdown_deadline_, wait_predicate)) LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else LOG_INFO(log, "Successfully waited all the parts"); } -std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) +std::set StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) { if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) { @@ -4027,26 +4032,25 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart } else { - LOG_TRACE(log_, "Fetching parts for replica {}", replica); - data_parts_on_replicas.emplace_back(format_version_); - for (const auto & part : parts) - { - if (data_parts_on_replicas.back().getContainingPart(part).empty()) - data_parts_on_replicas.back().add(part); - } + LOG_TRACE(log_, "Fetching parts for replica {}: [{}]", replica, fmt::join(parts, ", ")); + data_parts_on_replicas.emplace_back(format_version_, parts); } } - std::vector our_unique_parts; + if (data_parts_on_replicas.empty()) + { + LOG_TRACE(log_, "Has no active replicas, will no try to wait for fetch"); + return {}; + } + + std::set our_unique_parts; for (const auto & part : our_parts) { - LOG_TRACE(log_, "Looking for part {}", part); bool found = false; for (const auto & active_parts_set : data_parts_on_replicas) { if (!active_parts_set.getContainingPart(part).empty()) { - LOG_TRACE(log_, "Part {} found", part); found = true; break; } @@ -4054,8 +4058,8 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart if (!found) { - LOG_TRACE(log_, "Part not {} found", part); - our_unique_parts.emplace_back(MergeTreePartInfo::fromPartName(part, format_version_)); + LOG_TRACE(log_, "Part not {} found on other replicas", part); + our_unique_parts.emplace(MergeTreePartInfo::fromPartName(part, format_version_)); } } @@ -4836,9 +4840,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (shutdown_prepared_called.exchange(true)) return; - session_expired_callback_handler.reset(); - stopOutdatedDataPartsLoadingTask(); - + auto settings_ptr = getSettings(); /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); @@ -4850,39 +4852,17 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); - background_operations_assignee.finish(); - part_moves_between_shards_orchestrator.shutdown(); - - { - auto lock = queue.lockQueue(); - /// Cancel logs pulling after background task were cancelled. It's still - /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, - /// MUTATE, etc. query. - queue.pull_log_blocker.cancelForever(); - } - background_moves_assignee.finish(); - + shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } -void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) +void StorageReplicatedMergeTree::partialShutdown() { ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); partial_shutdown_called = true; partial_shutdown_event.set(); queue.notifySubscribersOnPartialShutdown(); - if (!part_of_full_shutdown) - { - /// If we are going to completely shutdown table we allow other - /// replicas to fetch parts which are unique for our replica. - /// - /// Replicas try to fetch part only in case the source replica is active, - /// so don't reset handler here. - LOG_DEBUG(log, "Reset active node, replica will be inactive"); - replica_is_active_node = nullptr; - } - else - LOG_DEBUG(log, "Will not reset active node, it will be reset completely during full shutdown"); + replica_is_active_node = nullptr; LOG_TRACE(log, "Waiting for threads to finish"); merge_selecting_task->deactivate(); @@ -4914,10 +4894,27 @@ void StorageReplicatedMergeTree::shutdown() flushAndPrepareForShutdown(); auto settings_ptr = getSettings(); - LOG_DEBUG(log, "Data parts exchange still exists {}", data_parts_exchange_endpoint != nullptr); - waitForUniquePartsToBeFetchedByOtherReplicas(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds()); + if (!shutdown_deadline.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); - replica_is_active_node = nullptr; + waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + + session_expired_callback_handler.reset(); + stopOutdatedDataPartsLoadingTask(); + + partialShutdown(); + + part_moves_between_shards_orchestrator.shutdown(); + background_operations_assignee.finish(); + + { + auto lock = queue.lockQueue(); + /// Cancel logs pulling after background task were cancelled. It's still + /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, + /// MUTATE, etc. query. + queue.pull_log_blocker.cancelForever(); + } + background_moves_assignee.finish(); auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 656e8df6ccb0..811a85240643 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -118,7 +118,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Partial shutdown called if we loose connection to zookeeper. /// Table can also recover after partial shutdown and continue /// to work. This method can be called regularly. - void partialShutdown(bool part_of_full_shutdown); + void partialShutdown(); /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). /// The shutdown process is split into two methods to make it more soft and fast. In database shutdown() @@ -368,15 +368,11 @@ class StorageReplicatedMergeTree final : public MergeTreeData ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; void addLastSentPart(const MergeTreePartInfo & info); - std::deque getLastSentParts() const - { - std::lock_guard lock(last_sent_parts_mutex); - return last_sent_parts; - } /// Wait required amount of milliseconds to give other replicas a chance to /// download unique parts from our replica - void waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms); + using ShutdownDeadline = std::chrono::time_point; + void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); private: std::atomic_bool are_restoring_replica {false}; @@ -483,6 +479,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData std::atomic shutdown_called {false}; std::atomic shutdown_prepared_called {false}; + std::optional shutdown_deadline; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; @@ -740,7 +738,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData */ String findReplicaHavingCoveringPart(LogEntry & entry, bool active); String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name); - static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); + static std::set findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. diff --git a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml new file mode 100644 index 000000000000..b23dbdc2607f --- /dev/null +++ b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -0,0 +1,5 @@ + + + 1000 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 50f2627d37c7..33d5c99202e2 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -57,6 +57,7 @@ ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/reverse_dns_query_function.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/disable_s3_env_credentials.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_wait_for_shutdown_replicated_tables.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/backups.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. From da61a8c509e2d43275f0c8f06dfd065f2874f79f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:16:00 +0200 Subject: [PATCH 101/226] Fix --- .../config.d/enable_wait_for_shutdown_replicated_tables.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml index b23dbdc2607f..504841296a85 100644 --- a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml +++ b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -1,5 +1,5 @@ - 1000 + 3000 From 8cdb181c3909802ae3bdd48fd118358b50ef027d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:52:31 +0200 Subject: [PATCH 102/226] Reduce logging level --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 88bd788b9efb..0f5a52b275c8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4000,7 +4000,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St std::unique_lock lock(last_sent_parts_mutex); if (!last_sent_parts_cv.wait_until(lock, shutdown_deadline_, wait_predicate)) - LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + LOG_INFO(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else LOG_INFO(log, "Successfully waited all the parts"); } From e618dd05cc73b7ad38296e7c28f66b6f077343f8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 7 Jul 2023 13:03:44 +0200 Subject: [PATCH 103/226] Fix clang tidy and race --- programs/server/Server.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 58cf3e5d2101..4b47da9affb0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2285,6 +2285,7 @@ void Server::updateServers( Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); std::vector all_servers; + all_servers.reserve(servers.size() + servers_to_start_before_tables.size()); for (auto & server : servers) all_servers.push_back(&server); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f5a52b275c8..504cf0326f06 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4890,14 +4890,23 @@ void StorageReplicatedMergeTree::shutdown() if (shutdown_called.exchange(true)) return; - if (!shutdown_prepared_called.load()) - flushAndPrepareForShutdown(); + flushAndPrepareForShutdown(); auto settings_ptr = getSettings(); if (!shutdown_deadline.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); - waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + try + { + waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + } + catch (const Exception & ex) + { + if (ex.code() == ErrorCodes::LOGICAL_ERROR) + throw; + + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } session_expired_callback_handler.reset(); stopOutdatedDataPartsLoadingTask(); @@ -4905,7 +4914,6 @@ void StorageReplicatedMergeTree::shutdown() partialShutdown(); part_moves_between_shards_orchestrator.shutdown(); - background_operations_assignee.finish(); { auto lock = queue.lockQueue(); From 3dd9c09579887d5627a2486b3e0cddcc15b2487d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jul 2023 11:25:48 +0000 Subject: [PATCH 104/226] Just save --- .../Optimizations/optimizePrewhere.cpp | 79 +++++++++++++++++++ .../MergeTreeBaseSelectProcessor.cpp | 2 + 2 files changed, 81 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ca8a412bf2ee..bcd3244b5a96 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -60,6 +61,74 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { +#ifdef WHATEVERSOMETHING +static void removeAliases(ActionsDAG * dag) +{ + using Node = ActionsDAG::Node; + struct Frame + { + const ActionsDAG::Node * node; + const ActionsDAG::Node * parent; + size_t next_child = 0; + }; + std::vector stack; + std::vector> aliases; + + /// collect aliases + auto output_nodes = dag->getOutputs(); + for (const auto * output_node : output_nodes) + { + stack.push_back({output_node, nullptr}); + while (!stack.empty()) + { + auto & frame = stack.back(); + const auto * parent = frame.parent; + const auto * node = frame.node; + + if (frame.next_child < node->children.size()) + { + auto next_frame = Frame{.node = node->children[frame.next_child], .parent = node}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } + + if (parent && node->type == ActionsDAG::ActionType::ALIAS) + aliases.emplace_back(const_cast(node), const_cast(parent)); + + stack.pop_back(); + } + } + + /// remove aliases from output nodes if any + for(auto it = output_nodes.begin(); it != output_nodes.end();) + { + if ((*it)->type == ActionsDAG::ActionType::ALIAS) + it = output_nodes.erase(it); + else + ++it; + } + + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "aliases found: {}", aliases.size()); + + /// disconnect aliases + for(auto [alias, parent]: aliases) + { + /// find alias in parent's children and replace it with alias child + for (auto & child : parent->children) + { + if (child == alias) + { + child = alias->children.front(); + break; + } + } + } + + /// remove aliases + dag->removeUnusedActions(); +} +#endif void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { @@ -162,6 +231,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "filter expression\n{}", filter_step->getExpression()->dumpDAG()); + auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), @@ -178,6 +249,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->need_filter = true; auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "prewhere_filter_actions\n{}", prewhere_filter_actions->dumpDAG()); + + // removeAliases(prewhere_filter_actions.get()); + // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "removeAliases\n{}", prewhere_filter_actions->dumpDAG()); ActionsChain actions_chain; @@ -260,7 +335,9 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->prewhere_column_name = prewere_filter_node_name; prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header BEFORE prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); read_from_merge_tree->updatePrewhereInfo(prewhere_info); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header AFTER prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); QueryPlan::Node * replace_old_filter_node = nullptr; bool remove_filter_node = false; @@ -321,10 +398,12 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) bool apply_match_step = false; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "read header\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); /// If column order does not match old filter step column order, match dag output nodes with header if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) { apply_match_step = true; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "rename_actions_dag\n{}", rename_actions_dag->dumpDAG()); matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 48adf36e6783..d3d8c0f2bc87 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -644,6 +644,7 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->prewhere_actions) { block = prewhere_info->prewhere_actions->updateHeader(std::move(block)); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "updateHeader()\n{}", block.dumpStructure()); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -654,6 +655,7 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->remove_prewhere_column) { + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "remove_column_name\n{}", prewhere_info->prewhere_column_name); block.erase(prewhere_info->prewhere_column_name); } else if (prewhere_info->need_filter) From 39a440fa0e99849d710e09bd031de5a52708fd6f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jul 2023 22:08:31 +0000 Subject: [PATCH 105/226] Build sort description based on original header --- .../QueryPlan/ReadFromMergeTree.cpp | 41 +++++++++++++++---- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 32bfa6935df7..3d1e26501887 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -139,17 +139,42 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) /// build sort description for output stream static void updateSortDescriptionForOutputStream( - DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info) + DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info, PrewhereInfoPtr prewhere_info) { + Block original_header = output_stream.header.cloneEmpty(); + /// build original header + if (prewhere_info && prewhere_info->prewhere_actions) + { + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + + for (auto & column : original_header) + { + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + { + LOG_DEBUG( + &Poco::Logger::get(__PRETTY_FUNCTION__), + "Found original column '{}' for '{}'", + original_node->result_name, + column.name); + column.name = original_node->result_name; + } + } + } + SortDescription sort_description; const Block & header = output_stream.header; - for (const auto & column_name : sorting_key_columns) + for (const auto & sorting_key : sorting_key_columns) { - if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) - == header.end()) + const auto it = std::find_if( + original_header.begin(), original_header.end(), [&sorting_key](const auto & column) { return column.name == sorting_key; }); + if (it == original_header.end()) break; - sort_description.emplace_back(column_name, sort_direction); + + const size_t column_pos = std::distance(original_header.begin(), it); + sort_description.emplace_back((header.begin() + column_pos)->name, sort_direction); } + if (!sort_description.empty()) { if (input_order_info) @@ -283,7 +308,8 @@ ReadFromMergeTree::ReadFromMergeTree( *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo()); + query_info.getInputOrderInfo(), + prewhere_info); } @@ -1575,7 +1601,8 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo()); + query_info.getInputOrderInfo(), + prewhere_info); } bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() From b2522d81c42cd19e2b6c76e18ee36ba7a0624fe7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Jun 2023 10:33:11 +0800 Subject: [PATCH 106/226] rewrite uniq to count --- .../RewriteUinqToCountVisitor.cpp | 88 +++++++++++++++++++ src/Interpreters/RewriteUinqToCountVisitor.h | 29 ++++++ 2 files changed, 117 insertions(+) create mode 100644 src/Interpreters/RewriteUinqToCountVisitor.cpp create mode 100644 src/Interpreters/RewriteUinqToCountVisitor.h diff --git a/src/Interpreters/RewriteUinqToCountVisitor.cpp b/src/Interpreters/RewriteUinqToCountVisitor.cpp new file mode 100644 index 000000000000..f491bb08c886 --- /dev/null +++ b/src/Interpreters/RewriteUinqToCountVisitor.cpp @@ -0,0 +1,88 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +static bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "" +} + +} + +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' +void RewriteUinqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) +{ + auto * selectq = ast->as(); + if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) + return; + auto expr_list = selectq->select(); + if (!expr_list || expr_list->children.size() != 1) + return; + auto * func = expr_list->children[0]->as(); + if (!func || (Poco::toLower(func->name) != "uniq" && Poco::toLower(func->name) != "uniqexact")) + return; + auto arg = func->arguments->children; + if (arg.size() != 1) + return; + if (!arg[0]->as()) + return; + if (selectq->tables()->as()->children[0]->as()->children.size() != 1) + return; + auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + if (!table_expr || table_expr->size() != 1 || !table_expr->database_and_table_name) + return; + // Check done, we now rewrite the AST + auto cloned_select_query = selectq->clone(); + expr_list->children[0] = makeASTFunction("count"); + + auto table_name = table_expr->database_and_table_name->as()->name(); + table_expr->children.clear(); + table_expr->children.emplace_back(std::make_shared()); + table_expr->database_and_table_name = nullptr; + table_expr->table_function = nullptr; + table_expr->subquery = table_expr->children[0]; + + auto column_name = arg[0]->as()->name(); + // Form AST for subquery + { + auto * select_ptr = cloned_select_query->as(); + select_ptr->refSelect()->children.clear(); + select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); + auto exprlist = std::make_shared(); + exprlist->children.emplace_back(std::make_shared(column_name)); + cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); + + auto expr = std::make_shared(); + expr->children.emplace_back(cloned_select_query); + auto select_with_union = std::make_shared(); + select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; + select_with_union->is_normalized = false; + select_with_union->list_of_modes.clear(); + select_with_union->set_of_modes.clear(); + select_with_union->children.emplace_back(expr); + select_with_union->list_of_selects = expr; + table_expr->children[0]->as()->children.emplace_back(select_with_union); + } +} + +} diff --git a/src/Interpreters/RewriteUinqToCountVisitor.h b/src/Interpreters/RewriteUinqToCountVisitor.h new file mode 100644 index 000000000000..d7aa745352b2 --- /dev/null +++ b/src/Interpreters/RewriteUinqToCountVisitor.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include "Interpreters/TreeRewriter.h" + +namespace DB +{ + +class ASTFunction; + +/// Simple rewrite: +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to +/// 'SELECT count() FROM (SELECT DISTINCT x ...)' +/// +/// 'SELECT uniq() FROM (SELECT x ... GROUP BY x)' to +/// 'SELECT count() FROM (SELECT x ... GROUP BY x)' +/// +/// Note we can rewrite all uniq variants except uniqUpTo. +class RewriteUinqToCountMatcher +{ +public: + struct Data {}; + static void visit(ASTPtr & ast, Data &); + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +using RewriteUinqToCountVisitor = InDepthNodeVisitor; +} From bcfc0db9f1bf1e6f8943687dcf1b122e94a86edd Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sun, 25 Jun 2023 14:43:39 +0800 Subject: [PATCH 107/226] add settiings --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 7 ++ .../RewriteUinqToCountVisitor.cpp | 88 ----------------- .../RewriteUniqToCountVisitor.cpp | 99 +++++++++++++++++++ ...tVisitor.h => RewriteUniqToCountVisitor.h} | 4 +- 5 files changed, 109 insertions(+), 90 deletions(-) delete mode 100644 src/Interpreters/RewriteUinqToCountVisitor.cpp create mode 100644 src/Interpreters/RewriteUniqToCountVisitor.cpp rename src/Interpreters/{RewriteUinqToCountVisitor.h => RewriteUniqToCountVisitor.h} (87%) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf07506..f5ab71a3b0d2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,6 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and other variants(except uniqUpTo) to distinct to to count, it is a RBO based optimization.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4d..ebde5df25337 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include @@ -426,6 +427,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr); } + if (settings.optimize_uniq_to_count) + { + RewriteUniqToCountMatcher::Data data_rewrite_uniq_count; + RewriteUniqToCountVisitor(data_rewrite_uniq_count).visit(query_ptr); + } + JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view); bool got_storage_from_query = false; diff --git a/src/Interpreters/RewriteUinqToCountVisitor.cpp b/src/Interpreters/RewriteUinqToCountVisitor.cpp deleted file mode 100644 index f491bb08c886..000000000000 --- a/src/Interpreters/RewriteUinqToCountVisitor.cpp +++ /dev/null @@ -1,88 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -static bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "" -} - -} - -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' -void RewriteUinqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) -{ - auto * selectq = ast->as(); - if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) - return; - auto expr_list = selectq->select(); - if (!expr_list || expr_list->children.size() != 1) - return; - auto * func = expr_list->children[0]->as(); - if (!func || (Poco::toLower(func->name) != "uniq" && Poco::toLower(func->name) != "uniqexact")) - return; - auto arg = func->arguments->children; - if (arg.size() != 1) - return; - if (!arg[0]->as()) - return; - if (selectq->tables()->as()->children[0]->as()->children.size() != 1) - return; - auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); - if (!table_expr || table_expr->size() != 1 || !table_expr->database_and_table_name) - return; - // Check done, we now rewrite the AST - auto cloned_select_query = selectq->clone(); - expr_list->children[0] = makeASTFunction("count"); - - auto table_name = table_expr->database_and_table_name->as()->name(); - table_expr->children.clear(); - table_expr->children.emplace_back(std::make_shared()); - table_expr->database_and_table_name = nullptr; - table_expr->table_function = nullptr; - table_expr->subquery = table_expr->children[0]; - - auto column_name = arg[0]->as()->name(); - // Form AST for subquery - { - auto * select_ptr = cloned_select_query->as(); - select_ptr->refSelect()->children.clear(); - select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); - auto exprlist = std::make_shared(); - exprlist->children.emplace_back(std::make_shared(column_name)); - cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); - - auto expr = std::make_shared(); - expr->children.emplace_back(cloned_select_query); - auto select_with_union = std::make_shared(); - select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; - select_with_union->is_normalized = false; - select_with_union->list_of_modes.clear(); - select_with_union->set_of_modes.clear(); - select_with_union->children.emplace_back(expr); - select_with_union->list_of_selects = expr; - table_expr->children[0]->as()->children.emplace_back(select_with_union); - } -} - -} diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp new file mode 100644 index 000000000000..c3a9fb7547ca --- /dev/null +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -0,0 +1,99 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +static bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta"; +} + +class PrintTreeMatcher +{ +public: + struct Data {String prefix;}; + static void visit(ASTPtr & ast, Data &) + { + ast->getID(); + } + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +using PrintTreeVisitor = InDepthNodeVisitor; + +} + +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' +void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) +{ + auto * selectq = ast->as(); + if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) + return; + auto expr_list = selectq->select(); + if (!expr_list || expr_list->children.size() != 1) + return; + auto * func = expr_list->children[0]->as(); + if (!func || !matchFnUniq(func->name)) + return; + if (selectq->tables()->as()->children[0]->as()->children.size() != 1) + return; + auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) + return; + auto * subquery = table_expr->subquery->as(); + subquery->formatForLogging(0); + + // Check done, we now rewrite the AST + auto cloned_select_query = selectq->clone(); + expr_list->children[0] = makeASTFunction("count"); + +// auto table_name = table_expr->database_and_table_name->as()->name(); + table_expr->children.clear(); + table_expr->children.emplace_back(std::make_shared()); + table_expr->database_and_table_name = nullptr; + table_expr->table_function = nullptr; + table_expr->subquery = table_expr->children[0]; + +// auto column_name = arg[0]->as()->name(); +// // Form AST for subquery +// { +// auto * select_ptr = cloned_select_query->as(); +// select_ptr->refSelect()->children.clear(); +// select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); +// auto exprlist = std::make_shared(); +// exprlist->children.emplace_back(std::make_shared(column_name)); +// cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); +// +// auto expr = std::make_shared(); +// expr->children.emplace_back(cloned_select_query); +// auto select_with_union = std::make_shared(); +// select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; +// select_with_union->is_normalized = false; +// select_with_union->list_of_modes.clear(); +// select_with_union->set_of_modes.clear(); +// select_with_union->children.emplace_back(expr); +// select_with_union->list_of_selects = expr; +// table_expr->children[0]->as()->children.emplace_back(select_with_union); +// } +} + +} diff --git a/src/Interpreters/RewriteUinqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h similarity index 87% rename from src/Interpreters/RewriteUinqToCountVisitor.h rename to src/Interpreters/RewriteUniqToCountVisitor.h index d7aa745352b2..f59206101d43 100644 --- a/src/Interpreters/RewriteUinqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -17,7 +17,7 @@ class ASTFunction; /// 'SELECT count() FROM (SELECT x ... GROUP BY x)' /// /// Note we can rewrite all uniq variants except uniqUpTo. -class RewriteUinqToCountMatcher +class RewriteUniqToCountMatcher { public: struct Data {}; @@ -25,5 +25,5 @@ class RewriteUinqToCountMatcher static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } }; -using RewriteUinqToCountVisitor = InDepthNodeVisitor; +using RewriteUniqToCountVisitor = InDepthNodeVisitor; } From 0dc1fdd67f8609564cda4ca61d17913d05d318c2 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 28 Jun 2023 14:12:24 +0800 Subject: [PATCH 108/226] add some comments --- .../RewriteUniqToCountVisitor.cpp | 111 ++++++++++-------- src/Interpreters/RewriteUniqToCountVisitor.h | 6 +- 2 files changed, 68 insertions(+), 49 deletions(-) diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index c3a9fb7547ca..d608b6dfae42 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -4,15 +4,11 @@ #include #include #include -#include #include #include #include #include -#include -#include #include -#include namespace DB @@ -21,28 +17,48 @@ namespace DB namespace { -static bool matchFnUniq(String func_name) +bool matchFnUniq(String func_name) { auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta"; + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" || name == "uniqCombined64"; } -class PrintTreeMatcher +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) { -public: - struct Data {String prefix;}; - static void visit(ASTPtr & ast, Data &) + if (!lhs || !rhs) + return false; + if (lhs->children.size() != rhs->children.size()) + return false; + for (size_t i = 0; i < lhs->children.size(); i++) { - ast->getID(); + if (lhs->children[i]->formatForLogging() != rhs->children[i]->formatForLogging()) // TODO not a elegant way + return false; } - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } -}; + return true; +} -using PrintTreeVisitor = InDepthNodeVisitor; +/// Test whether lhs contains all expr in rhs. +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs) +{ + if (!lhs || !rhs) + return false; + if (lhs->children.size() < rhs->children.size()) + return false; + std::vector lhs_strs; + for (const auto & le : lhs->children) + { + lhs_strs.emplace_back(le->formatForLogging()); + } + for (const auto & re : rhs->children) + { + if (std::find(lhs_strs.begin(), lhs_strs.end(), re->formatForLogging()) != lhs_strs.end()) + return false; + } + return true; +} } -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) { auto * selectq = ast->as(); @@ -60,40 +76,43 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) return; auto * subquery = table_expr->subquery->as(); - subquery->formatForLogging(0); + if (!subquery) + return; + auto * sub_selectq = subquery->children[0]->as()->children[0]->as()->children[0]->as(); + if (!sub_selectq) + return; - // Check done, we now rewrite the AST - auto cloned_select_query = selectq->clone(); - expr_list->children[0] = makeASTFunction("count"); + auto match_distinct = [&]() -> bool + { + if (!sub_selectq->distinct) + return false; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return false; + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as())) + return false; + return true; + }; -// auto table_name = table_expr->database_and_table_name->as()->name(); - table_expr->children.clear(); - table_expr->children.emplace_back(std::make_shared()); - table_expr->database_and_table_name = nullptr; - table_expr->table_function = nullptr; - table_expr->subquery = table_expr->children[0]; + auto match_group_by = [&]() -> bool + { + auto group_by = sub_selectq->groupBy(); + if (!group_by) + return false; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return false; + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), group_by->as())) + return false; + /// subquery select expression list must contain all columns in uniq expression list + expressionListContainsAll(sub_expr_list->as(), func->children[0]->as()); + return true; + }; -// auto column_name = arg[0]->as()->name(); -// // Form AST for subquery -// { -// auto * select_ptr = cloned_select_query->as(); -// select_ptr->refSelect()->children.clear(); -// select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); -// auto exprlist = std::make_shared(); -// exprlist->children.emplace_back(std::make_shared(column_name)); -// cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); -// -// auto expr = std::make_shared(); -// expr->children.emplace_back(cloned_select_query); -// auto select_with_union = std::make_shared(); -// select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; -// select_with_union->is_normalized = false; -// select_with_union->list_of_modes.clear(); -// select_with_union->set_of_modes.clear(); -// select_with_union->children.emplace_back(expr); -// select_with_union->list_of_selects = expr; -// table_expr->children[0]->as()->children.emplace_back(select_with_union); -// } + if (match_distinct() || match_group_by()) + expr_list->children[0] = makeASTFunction("count"); } } diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h index f59206101d43..42a86049bb96 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -10,11 +10,11 @@ namespace DB class ASTFunction; /// Simple rewrite: -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to +/// 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to /// 'SELECT count() FROM (SELECT DISTINCT x ...)' /// -/// 'SELECT uniq() FROM (SELECT x ... GROUP BY x)' to -/// 'SELECT count() FROM (SELECT x ... GROUP BY x)' +/// 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to +/// 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' /// /// Note we can rewrite all uniq variants except uniqUpTo. class RewriteUniqToCountMatcher From ca6905b308972f207b19e041149621d185913b3c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 7 Jul 2023 17:03:37 +0800 Subject: [PATCH 109/226] rewrite support alias --- src/Core/Settings.h | 2 +- .../RewriteUniqToCountVisitor.cpp | 97 +++++++++++++------ 2 files changed, 71 insertions(+), 28 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f5ab71a3b0d2..29ea7fe5cb91 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,7 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and other variants(except uniqUpTo) to distinct to to count, it is a RBO based optimization.", 0) + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause, it is a RBO based optimization.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index d608b6dfae42..587a905e4c53 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -14,16 +14,49 @@ namespace DB { +using Aliases = std::unordered_map; + namespace { bool matchFnUniq(String func_name) { auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" || name == "uniqCombined64"; + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; +} + +bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) +{ + if (lhs->getTreeHash() == rhs->getTreeHash()) + { + return true; + } + else + { + auto * lhs_idf = lhs->as(); + auto * rhs_idf = rhs->as(); + if (lhs_idf && rhs_idf) + { + /// compound identifiers, such as: + if (lhs_idf->shortName() == rhs_idf->shortName()) + return true; + + /// translate alias + if (alias.find(lhs_idf->shortName()) != alias.end()) + lhs_idf = alias.find(lhs_idf->shortName())->second->as(); + + if (alias.find(rhs_idf->shortName()) != alias.end()) + rhs_idf = alias.find(rhs_idf->shortName())->second->as(); + + if (lhs_idf->shortName() == rhs_idf->shortName()) + return true; + } + } + return false; } -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases & alias) { if (!lhs || !rhs) return false; @@ -31,27 +64,23 @@ bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) return false; for (size_t i = 0; i < lhs->children.size(); i++) { - if (lhs->children[i]->formatForLogging() != rhs->children[i]->formatForLogging()) // TODO not a elegant way + if (!expressionEquals(lhs->children[i], rhs->children[i], alias)) return false; } return true; } -/// Test whether lhs contains all expr in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs) +/// Test whether lhs contains all expressions in rhs. +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases alias) { if (!lhs || !rhs) return false; if (lhs->children.size() < rhs->children.size()) return false; - std::vector lhs_strs; - for (const auto & le : lhs->children) - { - lhs_strs.emplace_back(le->formatForLogging()); - } for (const auto & re : rhs->children) { - if (std::find(lhs_strs.begin(), lhs_strs.end(), re->formatForLogging()) != lhs_strs.end()) + auto predicate = [&re, &alias](ASTPtr & le) { return expressionEquals(le, re, alias); }; + if (std::find_if(lhs->children.begin(), lhs->children.end(), predicate) == lhs->children.end()) return false; } return true; @@ -72,46 +101,60 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return; if (selectq->tables()->as()->children[0]->as()->children.size() != 1) return; - auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + auto * table_expr = selectq->tables() + ->as() + ->children[0] + ->as() + ->children[0] + ->as(); if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) return; auto * subquery = table_expr->subquery->as(); if (!subquery) return; - auto * sub_selectq = subquery->children[0]->as()->children[0]->as()->children[0]->as(); + auto * sub_selectq = subquery->children[0] + ->as()->children[0] + ->as()->children[0] + ->as(); if (!sub_selectq) return; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return; - auto match_distinct = [&]() -> bool + /// collect subquery select expressions alias + std::unordered_map alias; + for (auto expr : sub_expr_list->children) + { + if (!expr->tryGetAlias().empty()) + alias.insert({expr->tryGetAlias(), expr}); + } + + auto match_subquery_with_distinct = [&]() -> bool { if (!sub_selectq->distinct) return false; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return false; /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as())) + if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as(), alias)) return false; return true; }; - auto match_group_by = [&]() -> bool + auto match_subquery_with_group_by = [&]() -> bool { - auto group_by = sub_selectq->groupBy(); + auto group_by = sub_selectq->groupBy(); // TODO group by type if (!group_by) return false; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), group_by->as())) + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), group_by->as(), alias)) return false; /// subquery select expression list must contain all columns in uniq expression list - expressionListContainsAll(sub_expr_list->as(), func->children[0]->as()); + if (!expressionListContainsAll(sub_expr_list->as(), func->children[0]->as(), alias)) + return false; return true; }; - if (match_distinct() || match_group_by()) + if (match_subquery_with_distinct() || match_subquery_with_group_by()) expr_list->children[0] = makeASTFunction("count"); } From e0b223aa7eac0e780a4048a2e302f67406ace0aa Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 7 Jul 2023 18:04:15 +0800 Subject: [PATCH 110/226] add tests for uniq to count rewrite --- .../test_rewrite_uniq_to_count/__init__.py | 0 .../test_rewrite_uniq_to_count/test.py | 81 +++++++++++++++++++ 2 files changed, 81 insertions(+) create mode 100644 tests/integration/test_rewrite_uniq_to_count/__init__.py create mode 100644 tests/integration/test_rewrite_uniq_to_count/test.py diff --git a/tests/integration/test_rewrite_uniq_to_count/__init__.py b/tests/integration/test_rewrite_uniq_to_count/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py new file mode 100644 index 000000000000..2e21a8a50a5c --- /dev/null +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -0,0 +1,81 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + prepare() + yield cluster + finally: + shutdown() + cluster.shutdown() + + +def prepare(): + node.query( + """ + CREATE TABLE IF NOT EXISTS test_rewrite_uniq_to_count + ( + `a` UInt8, + `b` UInt8, + `c` UInt8 + ) + ENGINE = MergeTree + ORDER BY `a` + """ + ) + node.query("INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')") + node.query("INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')") + node.query("INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')") + + +def shutdown(): + node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") + + +def check(query, result): + # old analyzer + query = query + " settings optimize_uniq_to_count = 1" + assert node.query(query) == f"{result}\n" + assert "count" in node.query("EXPLAIN SYNTAX " + query) + + # # new analyzer + # query = query + ", allow_experimental_analyzer = 1" + # assert node.query(query) == f"{result}\n" + # assert "count" in node.query("EXPLAIN QUERY_TREE " + query) + + +def test_rewrite_distinct(started_cluster): + check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", + 3) + + check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", + 3) + + check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", + 3) + + check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as n FROM test_rewrite_uniq_to_count) t", + 3) + + +def test_rewrite_group_by(started_cluster): + check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", + 3) + + check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3) + + check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3) + + check("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3) + + check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3) From 8004f37e972c9efc7cfc9d66a1157b020e7b6d89 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 10 Jul 2023 17:09:05 +0800 Subject: [PATCH 111/226] rewrite uniq to count in new analyzer --- src/Analyzer/Passes/QueryAnalysisPass.h | 1 + src/Analyzer/Passes/UniqToCountPass.cpp | 166 ++++++++++++++++++ src/Analyzer/Passes/UniqToCountPass.h | 30 ++++ src/Analyzer/QueryTreePassManager.cpp | 2 + src/Core/Settings.h | 2 +- .../RewriteUniqToCountVisitor.cpp | 12 +- src/Interpreters/RewriteUniqToCountVisitor.h | 17 +- .../test_rewrite_uniq_to_count/test.py | 39 +++- 8 files changed, 246 insertions(+), 23 deletions(-) create mode 100644 src/Analyzer/Passes/UniqToCountPass.cpp create mode 100644 src/Analyzer/Passes/UniqToCountPass.h diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index fa8778ebf762..ea845f26bd9e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -51,6 +51,7 @@ namespace DB * Function `arrayJoin` is handled properly. * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. * Replace `countDistinct` and `countIfDistinct` aggregate functions using setting count_distinct_implementation. + * Replace `uniq` and `uniq` and its variants(except uniqUpTo) into `count` aggregate functions using setting optimize_uniq_to_count. * Add -OrNull suffix to aggregate functions if setting aggregate_functions_null_for_empty is true. * Function `exists` is converted into `in`. * Functions `in`, `notIn`, `globalIn`, `globalNotIn` converted into `nullIn`, `notNullIn`, `globalNullIn`, `globalNotNullIn` if setting transform_null_in is true. diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp new file mode 100644 index 000000000000..1ffb83a6e364 --- /dev/null +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -0,0 +1,166 @@ +#include "UniqToCountPass.h" + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +using Aliases = std::unordered_map; + +namespace +{ + + bool matchFnUniq(String func_name) + { + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; + } + + bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs, const Aliases & alias) + { + auto * lhs_node = lhs->as(); + auto * rhs_node = rhs->as(); + + if (lhs_node && rhs_node) + { + if (lhs_node->getColumn() == rhs_node->getColumn()) + return true; + + /// translate alias + if (lhs->hasAlias() && alias.find(lhs->getAlias()) != alias.end()) + lhs_node = alias.find(lhs->getAlias())->second->as(); + + if (rhs->hasAlias() && alias.find(rhs->getAlias()) != alias.end()) + rhs_node = alias.find(rhs->getAlias())->second->as(); + + if (lhs_node && rhs_node && lhs_node == rhs_node) + return true; + } + return false; + } + + bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + { + if (lhs.size() != rhs.size()) + return false; + for (size_t i = 0; i < lhs.size(); i++) + { + if (!nodeEquals(lhs[i], rhs[i], alias)) + return false; + } + return true; + } + + bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + { + if (lhs.size() < rhs.size()) + return false; + for (const auto & re : rhs) + { + auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re, alias); }; + if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) + return false; + } + return true; + } + +} + +class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + if (!getSettings().optimize_uniq_to_count) + return; + + auto * query_node = node->as(); + if (!query_node) + return; + + /// Check that query has only single table expression which is subquery + auto * subquery_node = query_node->getJoinTree()->as(); + if (!subquery_node) + return; + + /// Check that query has only single node in projection + auto & projection_nodes = query_node->getProjection().getNodes(); + if (projection_nodes.size() != 1) + return; + + /// Check that projection_node is a function + auto & projection_node = projection_nodes[0]; + auto * function_node = projection_node->as(); + if (!function_node) + return; + + /// Check that query single projection node is `uniq` or its variants + if (!matchFnUniq(function_node->getFunctionName())) + return; + + /// collect subquery select expressions alias. + /// TODO new analyzer will lose alias info, so we will collect nothing and we can not rewrite SQL with alias. + Aliases alias; + for (auto & subquery_projection_node : subquery_node->getProjection().getNodes()) + { + if (subquery_projection_node->hasAlias()) + alias.insert({subquery_projection_node->getAlias(), subquery_projection_node}); + } + + auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); + + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' + auto match_subquery_with_distinct = [&]() -> bool + { + if (!subquery_node->isDistinct()) + return false; + /// uniq expression list == subquery group by expression list + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes(), alias)) + return false; + return true; + }; + + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' + auto match_subquery_with_group_by = [&]() -> bool + { + if (!subquery_node->hasGroupBy()) + return false; + /// uniq argument node list == subquery group by node list + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren(), alias)) + return false; + /// subquery select node list must contain all columns in uniq argument node list + if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes, alias)) + return false; + return true; + }; + + /// Replace uniq of initial query to count + if (match_subquery_with_distinct() || match_subquery_with_group_by()) + { + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + function_node->resolveAsAggregateFunction(std::move(aggregate_function)); + function_node->getArguments().getNodes().clear(); + query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); + } + } +}; + + +void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + UniqToCountVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/UniqToCountPass.h b/src/Analyzer/Passes/UniqToCountPass.h new file mode 100644 index 000000000000..4992d524e5e7 --- /dev/null +++ b/src/Analyzer/Passes/UniqToCountPass.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +namespace DB +{ + +/** Optimize `uniq` and its variants(except uniqUpTo) into `count` over subquery. + * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to + * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' + * + * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to + * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' + * + * Note that we can rewrite all uniq variants except uniqUpTo. + */ +class UniqToCountPass final : public IQueryTreePass +{ +public: + String getName() override { return "UniqToCount"; } + + String getDescription() override + { + return "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause."; + } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a6da2a666150..dd75b0f586d8 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -246,6 +247,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 29ea7fe5cb91..f71346be7b80 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,7 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause, it is a RBO based optimization.", 0) + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index 587a905e4c53..ac42a8a82dab 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -26,7 +26,7 @@ bool matchFnUniq(String func_name) || name == "uniqCombined64"; } -bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) +bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, const Aliases & alias) { if (lhs->getTreeHash() == rhs->getTreeHash()) { @@ -56,7 +56,7 @@ bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) return false; } -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases & alias) +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) { if (!lhs || !rhs) return false; @@ -71,7 +71,7 @@ bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Alia } /// Test whether lhs contains all expressions in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases alias) +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) { if (!lhs || !rhs) return false; @@ -123,13 +123,14 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return; /// collect subquery select expressions alias - std::unordered_map alias; + Aliases alias; for (auto expr : sub_expr_list->children) { if (!expr->tryGetAlias().empty()) alias.insert({expr->tryGetAlias(), expr}); } + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' auto match_subquery_with_distinct = [&]() -> bool { if (!sub_selectq->distinct) @@ -140,9 +141,10 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return true; }; + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' auto match_subquery_with_group_by = [&]() -> bool { - auto group_by = sub_selectq->groupBy(); // TODO group by type + auto group_by = sub_selectq->groupBy(); if (!group_by) return false; /// uniq expression list == subquery group by expression list diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h index 42a86049bb96..94528ccf2ee3 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -9,14 +9,15 @@ namespace DB class ASTFunction; -/// Simple rewrite: -/// 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to -/// 'SELECT count() FROM (SELECT DISTINCT x ...)' -/// -/// 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to -/// 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' -/// -/// Note we can rewrite all uniq variants except uniqUpTo. +/** Optimize `uniq` into `count` over subquery. + * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to + * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' + * + * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to + * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' + * + * Note that we can rewrite all uniq variants except uniqUpTo. + */ class RewriteUniqToCountMatcher { public: diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index 2e21a8a50a5c..af0c4f091179 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -42,40 +42,61 @@ def check(query, result): # old analyzer query = query + " settings optimize_uniq_to_count = 1" assert node.query(query) == f"{result}\n" - assert "count" in node.query("EXPLAIN SYNTAX " + query) + assert "count()" in node.query("EXPLAIN SYNTAX " + query) + + # new analyzer + query = query + ", allow_experimental_analyzer = 1" + assert node.query(query) == f"{result}\n" + assert "count()" in node.query("EXPLAIN QUERY TREE " + query) - # # new analyzer - # query = query + ", allow_experimental_analyzer = 1" - # assert node.query(query) == f"{result}\n" - # assert "count" in node.query("EXPLAIN QUERY_TREE " + query) + +# For new analyzer loses alias info, we can not rewrite SQL with alias. +def check_by_old_analyzer(query, result): + # only old analyzer + query = query + " settings optimize_uniq_to_count = 1" + assert node.query(query) == f"{result}\n" + assert "count()" in node.query("EXPLAIN SYNTAX " + query) def test_rewrite_distinct(started_cluster): + # simple test check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", 3) + # test subquery alias check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", 3) + # test table.column check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", 3) - check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as n FROM test_rewrite_uniq_to_count) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3) + + # test select expression alias + check_by_old_analyzer("SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3) def test_rewrite_group_by(started_cluster): + # simple test check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", 3) + # test subquery alias check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3) - check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3) - check("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3) - check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3) From a01a6834482cb8af82dd528fa557f3157b60eea9 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 09:51:52 +0800 Subject: [PATCH 112/226] fix style --- .../test_rewrite_uniq_to_count/test.py | 79 +++++++++++++------ 1 file changed, 53 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index af0c4f091179..ec9dc6d9b7fa 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -29,13 +29,21 @@ def prepare(): ORDER BY `a` """ ) - node.query("INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')") - node.query("INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')") - node.query("INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')") + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')" + ) + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')" + ) + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')" + ) def shutdown(): - node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") + node.query( + "DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC" + ) def check(query, result): @@ -50,7 +58,6 @@ def check(query, result): assert "count()" in node.query("EXPLAIN QUERY TREE " + query) -# For new analyzer loses alias info, we can not rewrite SQL with alias. def check_by_old_analyzer(query, result): # only old analyzer query = query + " settings optimize_uniq_to_count = 1" @@ -60,43 +67,63 @@ def check_by_old_analyzer(query, result): def test_rewrite_distinct(started_cluster): # simple test - check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", - 3) + check( + "SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", + 3, + ) # test subquery alias - check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", - 3) + check( + "SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", + 3, + ) - # test table.column - check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", - 3) + # test compound column name + check( + "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3) + check_by_old_analyzer( + "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3) + check_by_old_analyzer( + "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3, + ) def test_rewrite_group_by(started_cluster): # simple test - check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", - 3) + check( + "SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a)", + 3, + ) # test subquery alias - check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3) + check( + "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3, + ) From cd1111b17b07cdaade0e909a4139205763701d24 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 10:11:21 +0800 Subject: [PATCH 113/226] ignore alias for UniqToCountPass --- src/Analyzer/Passes/QueryAnalysisPass.h | 1 - src/Analyzer/Passes/UniqToCountPass.cpp | 90 +++++++------------ .../test_rewrite_uniq_to_count/test.py | 6 +- 3 files changed, 35 insertions(+), 62 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index ea845f26bd9e..fa8778ebf762 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -51,7 +51,6 @@ namespace DB * Function `arrayJoin` is handled properly. * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. * Replace `countDistinct` and `countIfDistinct` aggregate functions using setting count_distinct_implementation. - * Replace `uniq` and `uniq` and its variants(except uniqUpTo) into `count` aggregate functions using setting optimize_uniq_to_count. * Add -OrNull suffix to aggregate functions if setting aggregate_functions_null_for_empty is true. * Function `exists` is converted into `in`. * Functions `in`, `notIn`, `globalIn`, `globalNotIn` converted into `nullIn`, `notNullIn`, `globalNullIn`, `globalNotNullIn` if setting transform_null_in is true. diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp index 1ffb83a6e364..ae7952051e72 100644 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -11,65 +11,50 @@ namespace DB { -using Aliases = std::unordered_map; - namespace { - bool matchFnUniq(String func_name) - { - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; - } - - bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs, const Aliases & alias) - { - auto * lhs_node = lhs->as(); - auto * rhs_node = rhs->as(); - - if (lhs_node && rhs_node) - { - if (lhs_node->getColumn() == rhs_node->getColumn()) - return true; +bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; +} - /// translate alias - if (lhs->hasAlias() && alias.find(lhs->getAlias()) != alias.end()) - lhs_node = alias.find(lhs->getAlias())->second->as(); +bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs) +{ + auto * lhs_node = lhs->as(); + auto * rhs_node = rhs->as(); - if (rhs->hasAlias() && alias.find(rhs->getAlias()) != alias.end()) - rhs_node = alias.find(rhs->getAlias())->second->as(); + if (lhs_node && rhs_node && lhs_node->getColumn() == rhs_node->getColumn()) + return true; + return false; +} - if (lhs_node && rhs_node && lhs_node == rhs_node) - return true; - } +bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +{ + if (lhs.size() != rhs.size()) return false; - } - - bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + for (size_t i = 0; i < lhs.size(); i++) { - if (lhs.size() != rhs.size()) + if (!nodeEquals(lhs[i], rhs[i])) return false; - for (size_t i = 0; i < lhs.size(); i++) - { - if (!nodeEquals(lhs[i], rhs[i], alias)) - return false; - } - return true; } + return true; +} - bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) +bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +{ + if (lhs.size() < rhs.size()) + return false; + for (const auto & re : rhs) { - if (lhs.size() < rhs.size()) + auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re); }; + if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) return false; - for (const auto & re : rhs) - { - auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re, alias); }; - if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) - return false; - } - return true; } + return true; +} } @@ -108,15 +93,6 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContextgetFunctionName())) return; - /// collect subquery select expressions alias. - /// TODO new analyzer will lose alias info, so we will collect nothing and we can not rewrite SQL with alias. - Aliases alias; - for (auto & subquery_projection_node : subquery_node->getProjection().getNodes()) - { - if (subquery_projection_node->hasAlias()) - alias.insert({subquery_projection_node->getAlias(), subquery_projection_node}); - } - auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' @@ -125,7 +101,7 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContextisDistinct()) return false; /// uniq expression list == subquery group by expression list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes(), alias)) + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes())) return false; return true; }; @@ -136,10 +112,10 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContexthasGroupBy()) return false; /// uniq argument node list == subquery group by node list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren(), alias)) + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren())) return false; /// subquery select node list must contain all columns in uniq argument node list - if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes, alias)) + if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes)) return false; return true; }; diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index ec9dc6d9b7fa..d7fa9f394412 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -41,9 +41,7 @@ def prepare(): def shutdown(): - node.query( - "DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC" - ) + node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") def check(query, result): @@ -107,7 +105,7 @@ def test_rewrite_group_by(started_cluster): # test subquery alias check( "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, + 3, ) # test select expression alias From 063eebc16b8250b42f3f39b7cf00d8dcb578a702 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 12:59:34 +0800 Subject: [PATCH 114/226] fix clang-tidy checking --- src/Interpreters/RewriteUniqToCountVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index ac42a8a82dab..7445068207ab 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -124,7 +124,7 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) /// collect subquery select expressions alias Aliases alias; - for (auto expr : sub_expr_list->children) + for (const auto & expr : sub_expr_list->children) { if (!expr->tryGetAlias().empty()) alias.insert({expr->tryGetAlias(), expr}); From ce13131de47b28d61b013c386697bf011b13054e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 10:29:41 +0000 Subject: [PATCH 115/226] Fix integration tests --- src/Common/Config/ConfigReloader.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 45192d2d281f..12a449882d72 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -6,6 +6,7 @@ #include "ConfigProcessor.h" #include #include +#include namespace fs = std::filesystem; @@ -132,7 +133,8 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); #if USE_SSL - config_processor.decryptConfig(loaded_config); + if (endsWith(path, "config.xml")) + config_processor.decryptConfig(loaded_config); #endif /** We should remember last modification time if and only if config was successfully loaded From a73dca1c2ffd07c962801c6983bafa3a39d34317 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 13:27:11 +0000 Subject: [PATCH 116/226] Move getEncryptionMethod to CompressionCodecEncrypted.h --- src/Common/Config/ConfigProcessor.cpp | 37 +++++-------------- src/Compression/CompressionCodecEncrypted.cpp | 12 +++++- src/Compression/CompressionCodecEncrypted.h | 3 ++ 3 files changed, 24 insertions(+), 28 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 11b45977322c..9c2100b1c150 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -52,25 +52,6 @@ namespace ErrorCodes #endif } -#if USE_SSL -namespace -{ - -/// Get method for string name. Throw exception for wrong name -EncryptionMethod getEncryptionMethod(const std::string & name) -{ - if (name == "AES_128_GCM_SIV") - return AES_128_GCM_SIV; - else if (name == "AES_256_GCM_SIV") - return AES_256_GCM_SIV; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); -} - -} - -#endif - /// For cutting preprocessed path to this base static std::string main_config_path; @@ -204,11 +185,12 @@ static void mergeAttributes(Element & config_element, Element & with_element) std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { - auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + EncryptionMethod method = getEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(method); - DB::Memory<> memory; - memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); - auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); + Memory<> memory; + memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); auto encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); @@ -217,9 +199,10 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { - auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + EncryptionMethod method = getEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(method); - DB::Memory<> memory; + Memory<> memory; std::string encrypted_value; try @@ -232,7 +215,7 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const } memory.resize(codec.readDecompressedBlockSize(encrypted_value.data())); - codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); std::string decrypted_value = std::string(memory.data(), memory.size()); return decrypted_value; } @@ -797,7 +780,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 022bbd583e46..01bf6281a0e2 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -28,6 +28,17 @@ namespace DB namespace ErrorCodes { extern const int OPENSSL_ERROR; + extern const int BAD_ARGUMENTS; +} + +EncryptionMethod getEncryptionMethod(const std::string & name) +{ + if (name == "AES_128_GCM_SIV") + return AES_128_GCM_SIV; + else if (name == "AES_256_GCM_SIV") + return AES_256_GCM_SIV; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); } namespace @@ -79,7 +90,6 @@ namespace ErrorCodes { extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; } diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index 0f680dbcb090..a8faf3d0b6c1 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,6 +18,9 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; +/// Get method for string name. Throw exception for wrong name +EncryptionMethod getEncryptionMethod(const std::string & name); + /** This codec encrypts and decrypts blocks with AES-128 in * GCM-SIV mode (RFC-8452), which is the only cipher currently * supported. Although it is implemented as a compression codec From 3b8ecb172f81bcaa7538f257a3431cd4e0787645 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 14:43:25 +0000 Subject: [PATCH 117/226] Move descryption code to savePreprocessedConfig --- src/Common/Config/ConfigProcessor.cpp | 13 +++++++++---- src/Common/Config/ConfigProcessor.h | 12 ++++++------ src/Common/Config/ConfigReloader.cpp | 6 ------ src/Daemon/BaseDaemon.cpp | 4 ---- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 9c2100b1c150..0d40f782be26 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -183,7 +183,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) #if USE_SSL -std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) const { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -197,7 +197,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const return hex_value; } -std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) const { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -778,7 +778,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( #if USE_SSL -void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) +void ConfigProcessor::decryptEncryptedElements(LoadedConfig & loaded_config) { CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); @@ -788,7 +788,7 @@ void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) #endif -void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) +void ConfigProcessor::savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir) { try { @@ -843,6 +843,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, { LOG_WARNING(log, "Couldn't save preprocessed config to {}: {}", preprocessed_path, e.displayText()); } + +#if USE_SSL + if (fs::path(preprocessed_path).filename() == "config.xml") + decryptEncryptedElements(loaded_config); +#endif } void ConfigProcessor::setConfigPath(const std::string & config_path) diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 8d7caa9e9c8c..0da304bc0d25 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -94,18 +94,15 @@ class ConfigProcessor #if USE_SSL /// Encrypt text value - std::string encryptValue(const std::string & codec_name, const std::string & value); + std::string encryptValue(const std::string & codec_name, const std::string & value) const; /// Decrypt value - std::string decryptValue(const std::string & codec_name, const std::string & value); - - /// Decrypt nodes in config with specified encryption attributes - void decryptConfig(LoadedConfig & loaded_config); + std::string decryptValue(const std::string & codec_name, const std::string & value) const; #endif /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ - void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir); + void savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir); /// Set path of main config.xml. It will be cut from all configs placed to preprocessed_configs/ static void setConfigPath(const std::string & config_path); @@ -137,6 +134,9 @@ class ConfigProcessor #if USE_SSL void decryptRecursive(Poco::XML::Node * config_root); + + /// Decrypt elements in config with specified encryption attributes + void decryptEncryptedElements(LoadedConfig & loaded_config); #endif void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 12a449882d72..de7011b67bfa 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -6,7 +6,6 @@ #include "ConfigProcessor.h" #include #include -#include namespace fs = std::filesystem; @@ -132,11 +131,6 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); -#if USE_SSL - if (endsWith(path, "config.xml")) - config_processor.decryptConfig(loaded_config); -#endif - /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: * File is contain raw (and non-valid) data, therefore config is not applied. diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 3e31ed167c06..319d2bc8b5ba 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -763,10 +763,6 @@ void BaseDaemon::initialize(Application & self) ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); -#if USE_SSL - DB::ConfigProcessor(config_path).decryptConfig(loaded_config); -#endif - /// Write core dump on crash. { struct rlimit rlim; From b9adb2039bfdd90eb5b05b3f76a4aebd5e0b1757 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 15:13:47 +0000 Subject: [PATCH 118/226] Update MD docs --- docs/en/operations/configuration-files.md | 10 ++++------ docs/ru/operations/configuration-files.md | 10 ++++------ 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index f8f249598cc4..8cd34cc6c303 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -65,9 +65,11 @@ XML substitution example: Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. -## Decryption {#decryption} +## Encrypting Configuration {#encryption} -Elements with text nodes may be encrypted with [encryption codecs](../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. +You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec`` with the name of the encryption codec as value to the element to encrypt. + +Unlike attributes `from_zk`, `from_env` and `incl`` (or element `include``), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. Example: @@ -97,10 +99,6 @@ Example: 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` -:::note -The decryption is executed after creation of preprocessed configuration file. It means that elements with `encryption_codec` attribute in the preprocessed configuration file are encrypted. But the values of corresponding parameters in server's memory are decrypted. -::: - ## User Settings {#user-settings} The `config.xml` file can specify a separate config with user settings, profiles, and quotas. The relative path to this config is set in the `users_config` element. By default, it is `users.xml`. If `users_config` is omitted, the user settings, profiles, and quotas are specified directly in `config.xml`. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index df50d9009197..91877bdef952 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -85,9 +85,11 @@ $ cat /etc/clickhouse-server/users.d/alice.xml Сервер следит за изменениями конфигурационных файлов, а также файлов и ZooKeeper-узлов, которые были использованы при выполнении подстановок и переопределений, и перезагружает настройки пользователей и кластеров на лету. То есть, можно изменять кластера, пользователей и их настройки без перезапуска сервера. -## Расшифровка {#decryption} +## Шифрование {#encryption} -Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../sql-reference/statements/create/table.md#create-query-encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec`` с именем кодека шифрования как значение к элементу, который надо зашифровать. + +В отличии от аттрибутов `from_zk`, `from_env` и `incl`` (или элемента `include``), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. Пример: @@ -117,10 +119,6 @@ $ cat /etc/clickhouse-server/users.d/alice.xml 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` -:::note -Расшифровка выполняется после создания конфигурационного файла предобработки. Это означает что элементы с аттрибутом `encryption_codec` в конфигурационном файле предобработки зашифрованы. Но значения соответствующих параметров в памяти сервера расшифрованы. -::: - ## Примеры записи конфигурации на YAML {#example} Здесь можно рассмотреть пример реальной конфигурации записанной на YAML: [config.yaml.example](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.yaml.example). From c6c6389f3f64024a096b8df06850a1d4273a36ba Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 11:57:18 +0000 Subject: [PATCH 119/226] Add queries with explicitly enabled analyzer --- .../Optimizations/optimizePrewhere.cpp | 80 ------------------- ...ct_in_order_optimization_explain.reference | 31 +++++++ ..._distinct_in_order_optimization_explain.sh | 28 +++++-- 3 files changed, 54 insertions(+), 85 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index bcd3244b5a96..d9e98a9f41c0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB { @@ -61,75 +60,6 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { -#ifdef WHATEVERSOMETHING -static void removeAliases(ActionsDAG * dag) -{ - using Node = ActionsDAG::Node; - struct Frame - { - const ActionsDAG::Node * node; - const ActionsDAG::Node * parent; - size_t next_child = 0; - }; - std::vector stack; - std::vector> aliases; - - /// collect aliases - auto output_nodes = dag->getOutputs(); - for (const auto * output_node : output_nodes) - { - stack.push_back({output_node, nullptr}); - while (!stack.empty()) - { - auto & frame = stack.back(); - const auto * parent = frame.parent; - const auto * node = frame.node; - - if (frame.next_child < node->children.size()) - { - auto next_frame = Frame{.node = node->children[frame.next_child], .parent = node}; - ++frame.next_child; - stack.push_back(next_frame); - continue; - } - - if (parent && node->type == ActionsDAG::ActionType::ALIAS) - aliases.emplace_back(const_cast(node), const_cast(parent)); - - stack.pop_back(); - } - } - - /// remove aliases from output nodes if any - for(auto it = output_nodes.begin(); it != output_nodes.end();) - { - if ((*it)->type == ActionsDAG::ActionType::ALIAS) - it = output_nodes.erase(it); - else - ++it; - } - - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "aliases found: {}", aliases.size()); - - /// disconnect aliases - for(auto [alias, parent]: aliases) - { - /// find alias in parent's children and replace it with alias child - for (auto & child : parent->children) - { - if (child == alias) - { - child = alias->children.front(); - break; - } - } - } - - /// remove aliases - dag->removeUnusedActions(); -} -#endif - void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { if (stack.size() < 3) @@ -231,8 +161,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "filter expression\n{}", filter_step->getExpression()->dumpDAG()); - auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), @@ -249,10 +177,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->need_filter = true; auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "prewhere_filter_actions\n{}", prewhere_filter_actions->dumpDAG()); - - // removeAliases(prewhere_filter_actions.get()); - // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "removeAliases\n{}", prewhere_filter_actions->dumpDAG()); ActionsChain actions_chain; @@ -335,9 +259,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->prewhere_column_name = prewere_filter_node_name; prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header BEFORE prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); read_from_merge_tree->updatePrewhereInfo(prewhere_info); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header AFTER prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); QueryPlan::Node * replace_old_filter_node = nullptr; bool remove_filter_node = false; @@ -398,12 +320,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) bool apply_match_step = false; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "read header\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); /// If column order does not match old filter step column order, match dag output nodes with header if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) { apply_match_step = true; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "rename_actions_dag\n{}", rename_actions_dag->dumpDAG()); matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); } diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 0a123a2a50f1..3cee6da3c3d9 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -56,6 +56,7 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query +-- disable new analyzer -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a ASC Sorting (Stream): a ASC @@ -80,3 +81,33 @@ Sorting (Stream): a DESC, b DESC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC +-- enable new analyzer +-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns +Sorting (Stream): a_1 ASC +Sorting (Stream): a_1 ASC +Sorting (Stream): a_1 ASC +Sorting (Stream): a ASC +-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a ASC, b ASC +-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a DESC, b DESC +-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a ASC, b ASC +-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a DESC, b DESC +-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a ASC, b ASC diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 46919ae49b25..4c4e660030e0 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -81,18 +81,36 @@ $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain plan sorting=1 select disti echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" +echo "-- disable new analyzer" +DISABLE_ANALYZER="set allow_experimental_analyzer=0" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +echo "-- enable new analyzer" +ENABLE_ANALYZER="set allow_experimental_analyzer=1" +echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES + +echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES + $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" From 62e6a53d1caefa6d907f81d9151ffd930e3242d4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 12:16:09 +0000 Subject: [PATCH 120/226] Remove debug traces --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index d3d8c0f2bc87..48adf36e6783 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -644,7 +644,6 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->prewhere_actions) { block = prewhere_info->prewhere_actions->updateHeader(std::move(block)); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "updateHeader()\n{}", block.dumpStructure()); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -655,7 +654,6 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->remove_prewhere_column) { - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "remove_column_name\n{}", prewhere_info->prewhere_column_name); block.erase(prewhere_info->prewhere_column_name); } else if (prewhere_info->need_filter) From 9738d248c31feb4944339eb13162d1a24e7db44f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 12:17:48 +0000 Subject: [PATCH 121/226] Revert unnecessary changes --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index d9e98a9f41c0..ca8a412bf2ee 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -60,6 +60,7 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { + void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { if (stack.size() < 3) From 4e188a637979516df49212ac2b2043efdf16feaa Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 14 Jul 2023 08:39:37 -0400 Subject: [PATCH 122/226] Update s3.md regarding partitioned data --- .../engines/table-engines/integrations/s3.md | 21 +++---------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 0e2b48ef6a69..051945538b2d 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -57,7 +57,8 @@ Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter tok :::note As shown in the example, querying from S3 tables that are partitioned is -not directly supported at this time, but can be accomplished by querying the bucket contents with a wildcard. +not directly supported at this time, but can be accomplished by querying the individual partitions +using the S3 table function. The primary use-case for writing partitioned data in S3 is to enable transferring that data into another @@ -127,23 +128,7 @@ FROM s3('http://minio:10000/clickhouse//test_45.csv', 'minioadmin', 'minioadminp └────┴────┴────┘ ``` -#### Select from all partitions - -```sql -SELECT * -FROM s3('http://minio:10000/clickhouse//**', 'minioadmin', 'minioadminpassword', 'CSV') -``` -```response -┌─c1─┬─c2─┬─c3─┐ -│ 3 │ 2 │ 1 │ -└────┴────┴────┘ -┌─c1─┬─c2─┬─c3─┐ -│ 1 │ 2 │ 3 │ -└────┴────┴────┘ -┌─c1─┬─c2─┬─c3─┐ -│ 78 │ 43 │ 45 │ -└────┴────┴────┘ -``` +#### Limitation You may naturally try to `Select * from p`, but as noted above, this query will fail; use the preceding query. From feebad3d3bf8bb26e3df77845308c27126184403 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 14 Jul 2023 14:39:24 +0000 Subject: [PATCH 123/226] Try to fix tests by adding mutex like in onProgress() --- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 544442c4c05d..a05efa7ec3fd 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -176,6 +176,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) void WriteBufferFromHTTPServerResponse::onMemoryUsage(Int64 usage) { + std::lock_guard lock(mutex); peak_memory_usage = usage; } From 386adfad3365d3026ec8a3fe11536eead780262d Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 15 Jul 2023 16:21:58 +0000 Subject: [PATCH 124/226] Avro input format support Union with single type --- .../Formats/Impl/AvroRowInputFormat.cpp | 28 ++++++++++++++---- .../02813_avro_union_with_one_type.reference | 5 ++++ .../02813_avro_union_with_one_type.sh | 13 ++++++++ .../0_stateless/data_avro/union_one_type.avro | Bin 0 -> 304 bytes 4 files changed, 41 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02813_avro_union_with_one_type.reference create mode 100755 tests/queries/0_stateless/02813_avro_union_with_one_type.sh create mode 100644 tests/queries/0_stateless/data_avro/union_one_type.avro diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 4cd73cb23b59..771247a983bf 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -367,14 +367,25 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro break; case avro::AVRO_UNION: { - if (root_node->leaves() == 2 + if (root_node->leaves() == 1) + { + auto nested_deserialize = createDeserializeFn(root_node->leafAt(0), target_type); + return [nested_deserialize](IColumn & column, avro::Decoder & decoder) + { + decoder.decodeUnionIndex(); + nested_deserialize(column, decoder); + return true; + }; + } + /// FIXME Support UNION has more than two datatypes. + else if ( + root_node->leaves() == 2 && (root_node->leafAt(0)->type() == avro::AVRO_NULL || root_node->leafAt(1)->type() == avro::AVRO_NULL)) { int non_null_union_index = root_node->leafAt(0)->type() == avro::AVRO_NULL ? 1 : 0; if (target.isNullable()) { - auto nested_deserialize = this->createDeserializeFn( - root_node->leafAt(non_null_union_index), removeNullable(target_type)); + auto nested_deserialize = createDeserializeFn(root_node->leafAt(non_null_union_index), removeNullable(target_type)); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { ColumnNullable & col = assert_cast(column); @@ -393,7 +404,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro } else if (null_as_default) { - auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); + auto nested_deserialize = createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { int union_index = static_cast(decoder.decodeUnionIndex()); @@ -1169,12 +1180,19 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) case avro::Type::AVRO_NULL: return std::make_shared(); case avro::Type::AVRO_UNION: - if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) + if (node->leaves() == 1) + { + return avroNodeToDataType(node->leafAt(0)); + } + else if ( + node->leaves() == 2 + && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) { int nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0; auto nested_type = avroNodeToDataType(node->leafAt(nested_leaf_index)); return nested_type->canBeInsideNullable() ? makeNullable(nested_type) : nested_type; } + /// FIXME Support UNION has more than two datatypes. throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro type UNION is not supported for inserting."); case avro::Type::AVRO_SYMBOLIC: return avroNodeToDataType(avro::resolveSymbol(node)); diff --git a/tests/queries/0_stateless/02813_avro_union_with_one_type.reference b/tests/queries/0_stateless/02813_avro_union_with_one_type.reference new file mode 100644 index 000000000000..c65bed480550 --- /dev/null +++ b/tests/queries/0_stateless/02813_avro_union_with_one_type.reference @@ -0,0 +1,5 @@ +name String +favorite_number Int32 +favorite_color String +Alyssa 256 yellow +Ben 7 red diff --git a/tests/queries/0_stateless/02813_avro_union_with_one_type.sh b/tests/queries/0_stateless/02813_avro_union_with_one_type.sh new file mode 100755 index 000000000000..b58dc9126da9 --- /dev/null +++ b/tests/queries/0_stateless/02813_avro_union_with_one_type.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_DIR=$CUR_DIR/data_avro + +$CLICKHOUSE_LOCAL -q "desc file('$DATA_DIR/union_one_type.avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_DIR/union_one_type.avro')" diff --git a/tests/queries/0_stateless/data_avro/union_one_type.avro b/tests/queries/0_stateless/data_avro/union_one_type.avro new file mode 100644 index 0000000000000000000000000000000000000000..07e6140e5e2f020fd44f59a674598d7c4c7e7214 GIT binary patch literal 304 zcmeZI%3@>@Nh~YM*GtY%NloU+E6vFf1M`cMGg5OCXE9eRl~fj_Dp@Hg6{RNU7o{la zC@AG6=7L2+i&KkW{NjSdWUydrMPhD2PO2Wr6p&aG4L_46{V)2D+d7n$7llp literal 0 HcmV?d00001 From aef9575c3ee8f0f30070a44b5ff1867a014f1b03 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 15 Jul 2023 20:18:39 +0000 Subject: [PATCH 125/226] Fix test --- ...stinct_in_order_optimization_explain.reference | 9 +++++++-- ...2317_distinct_in_order_optimization_explain.sh | 15 ++++++++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 3cee6da3c3d9..21ce47cc6855 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -51,12 +51,12 @@ MergeTreeInOrder MergeTreeInOrder -- enabled, only part of distinct columns form prefix of sorting key MergeTreeThread +=== disable new analyzer === -- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query --- disable new analyzer -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a ASC Sorting (Stream): a ASC @@ -81,7 +81,12 @@ Sorting (Stream): a DESC, b DESC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC --- enable new analyzer +=== enable new analyzer === +-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a_1 ASC Sorting (Stream): a_1 ASC diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 4c4e660030e0..db8fa7c16002 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -76,13 +76,14 @@ $CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTI echo "-- enabled, only part of distinct columns form prefix of sorting key" $CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +echo "=== disable new analyzer ===" +DISABLE_ANALYZER="set allow_experimental_analyzer=0" + echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" -echo "-- disable new analyzer" -DISABLE_ANALYZER="set allow_experimental_analyzer=0" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" @@ -95,10 +96,14 @@ echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwr $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES -echo "-- enable new analyzer" +echo "=== enable new analyzer ===" ENABLE_ANALYZER="set allow_experimental_analyzer=1" + +echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES + echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" $CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" From 8f3469e7ad623b34c6272ba7c0cd132107034481 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 15 Jul 2023 20:18:57 +0000 Subject: [PATCH 126/226] Remove test from broken tests --- tests/analyzer_tech_debt.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa8..b420149418fb 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -82,7 +82,6 @@ 02242_join_rocksdb 02267_join_dup_columns_issue36199 02302_s3_file_pruning -02317_distinct_in_order_optimization_explain 02341_global_join_cte 02345_implicit_transaction 02352_grouby_shadows_arg From 315f4afe11d2a297f7b444f29366ba006c0127aa Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 05:20:09 +0000 Subject: [PATCH 127/226] Add any_value as a compatibility alias for any --- src/AggregateFunctions/AggregateFunctionAny.cpp | 1 + tests/queries/0_stateless/02813_any_value.reference | 2 ++ tests/queries/0_stateless/02813_any_value.sql | 2 ++ 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02813_any_value.reference create mode 100644 tests/queries/0_stateless/02813_any_value.sql diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index 7f57062126bf..fc8f50efabec 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -49,6 +49,7 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("any", { createAggregateFunctionAny, properties }); + factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("anyLast", { createAggregateFunctionAnyLast, properties }); factory.registerFunction("anyHeavy", { createAggregateFunctionAnyHeavy, properties }); diff --git a/tests/queries/0_stateless/02813_any_value.reference b/tests/queries/0_stateless/02813_any_value.reference new file mode 100644 index 000000000000..aa47d0d46d47 --- /dev/null +++ b/tests/queries/0_stateless/02813_any_value.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02813_any_value.sql b/tests/queries/0_stateless/02813_any_value.sql new file mode 100644 index 000000000000..d2c2324349c7 --- /dev/null +++ b/tests/queries/0_stateless/02813_any_value.sql @@ -0,0 +1,2 @@ +select any_value(number) from numbers(10); +select aNy_VaLue(number) from numbers(10); From 1eaa387da7bc0cfa17dc9b666f38dff6250595c2 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 05:23:15 +0000 Subject: [PATCH 128/226] update doc --- docs/en/sql-reference/aggregate-functions/reference/any.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index db19f524b312..f79fe66c05d7 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -12,3 +12,5 @@ To get a determinate result, you can use the ‘min’ or ‘max’ function ins In some cases, you can rely on the order of execution. This applies to cases when SELECT comes from a subquery that uses ORDER BY. When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. + +- Alias: `any_value` From 43cd600e21b4e3ac0d38e6be1daa153f3e9765ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 17:35:51 +0300 Subject: [PATCH 129/226] Update 02813_any_value.sql --- tests/queries/0_stateless/02813_any_value.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02813_any_value.sql b/tests/queries/0_stateless/02813_any_value.sql index d2c2324349c7..6bd2b66fde18 100644 --- a/tests/queries/0_stateless/02813_any_value.sql +++ b/tests/queries/0_stateless/02813_any_value.sql @@ -1,2 +1,3 @@ +SET max_block_size = 10, max_threads = 1; select any_value(number) from numbers(10); select aNy_VaLue(number) from numbers(10); From c443c798b120b0cc6ab2a33c2c4cdb78c056508b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sun, 16 Jul 2023 17:22:01 -0400 Subject: [PATCH 130/226] upgrade qpl to v1.2.0 and libaccel to v4.0 --- contrib/idxd-config | 2 +- contrib/qpl | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/idxd-config b/contrib/idxd-config index f6605c41a735..a836ce0e4205 160000 --- a/contrib/idxd-config +++ b/contrib/idxd-config @@ -1 +1 @@ -Subproject commit f6605c41a735e3fdfef2d2d18655a33af6490b99 +Subproject commit a836ce0e42052a69bffbbc14239ab4097f3b77f1 diff --git a/contrib/qpl b/contrib/qpl index 3f8f5cea2773..faaf19350459 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit 3f8f5cea27739f5261e8fd577dc233ffe88bf679 +Subproject commit faaf19350459c076e66bb5df11743c3fade59b73 From 7716479a37559afefc285191f780f1f424460826 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jul 2023 10:33:38 +0800 Subject: [PATCH 131/226] add comments for https://github.com/ClickHouse/ClickHouse/pull/52112 --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 899b84cc1324..ad199abaf8f9 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -233,6 +233,8 @@ namespace DB checkStatus(components_status, nested_column->getName(), format_name); /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column + /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. + /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From 8ea335aca7a102d5d06114799c2ada616b894ea8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jul 2023 10:43:13 +0800 Subject: [PATCH 132/226] update style --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index ad199abaf8f9..f688efa3290c 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -233,8 +233,8 @@ namespace DB checkStatus(components_status, nested_column->getName(), format_name); /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column - /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. - /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. + /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. + /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From 2bc7bb062cb5f7d99844f40f0735ff65575ab6d5 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 10:20:58 +0000 Subject: [PATCH 133/226] Add peak to http progress; not display peak if old server --- src/Common/ProgressIndication.cpp | 5 +++- src/Common/ProgressIndication.h | 6 ++-- .../WriteBufferFromHTTPServerResponse.cpp | 28 +++++++------------ .../HTTP/WriteBufferFromHTTPServerResponse.h | 7 ++--- src/Server/HTTPHandler.cpp | 6 ++-- 5 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index c9068720f8fb..960d864660cb 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -99,8 +99,11 @@ void ProgressIndication::writeFinalProgress() if (elapsed_ns) std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., " << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; + else + std::cout << ". "; auto peak_memory_usage = getMemoryUsage().peak; - std::cout << ".\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; + if (peak_memory_usage >= 0) + std::cout << "\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index e3c73e8e7e58..a12c025614e8 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -22,7 +22,9 @@ struct ThreadEventData UInt64 user_ms = 0; UInt64 system_ms = 0; UInt64 memory_usage = 0; - UInt64 peak_memory_usage = 0; + + // -1 used as flag 'is not show for old servers' + Int64 peak_memory_usage = -1; }; using HostToTimesMap = std::unordered_map; @@ -65,7 +67,7 @@ class ProgressIndication { UInt64 total = 0; UInt64 max = 0; - UInt64 peak = 0; + Int64 peak = -1; }; MemoryUsage getMemoryUsage() const; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index a05efa7ec3fd..9da24cfd9cb1 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -29,7 +29,7 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() } } -void WriteBufferFromHTTPServerResponse::writeHeaderSummary() +void WriteBufferFromHTTPServerResponse::writeHeaderProgressImpl(const char * header_name) { if (headers_finished_sending) return; @@ -43,19 +43,17 @@ void WriteBufferFromHTTPServerResponse::writeHeaderSummary() writeCString("\"}", progress_string_writer); if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << "\r\n" << std::flush; + *response_header_ostr << header_name << progress_string_writer.str() << "\r\n" << std::flush; } -void WriteBufferFromHTTPServerResponse::writeHeaderProgress() +void WriteBufferFromHTTPServerResponse::writeHeaderSummary() { - if (headers_finished_sending) - return; - - WriteBufferFromOwnString progress_string_writer; - accumulated_progress.writeJSON(progress_string_writer); + writeHeaderProgressImpl("X-ClickHouse-Summary: "); +} - if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Progress: " << progress_string_writer.str() << "\r\n" << std::flush; +void WriteBufferFromHTTPServerResponse::writeHeaderProgress() +{ + writeHeaderProgressImpl("X-ClickHouse-Progress: "); } void WriteBufferFromHTTPServerResponse::writeExceptionCode() @@ -154,7 +152,7 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( } -void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) +void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress, Int64 peak_memory_usage_) { std::lock_guard lock(mutex); @@ -163,7 +161,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) return; accumulated_progress.incrementPiecewiseAtomically(progress); - + peak_memory_usage = peak_memory_usage_; if (send_progress && progress_watch.elapsed() >= send_progress_interval_ms * 1000000) { progress_watch.restart(); @@ -174,12 +172,6 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) } } -void WriteBufferFromHTTPServerResponse::onMemoryUsage(Int64 usage) -{ - std::lock_guard lock(mutex); - peak_memory_usage = usage; -} - WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index f849a685aec6..b4c66357d3b3 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -43,10 +43,7 @@ class WriteBufferFromHTTPServerResponse final : public BufferWithOwnMemoryonProgress(progress); - auto thread_group = CurrentThread::getGroup(); - auto peak_memory_usage = thread_group->memory_tracker.getPeak(); - used_output.out->onMemoryUsage(peak_memory_usage); + const auto& thread_group = CurrentThread::getGroup(); + used_output.out->onProgress(progress, thread_group->memory_tracker.getPeak()); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) From 1d4d829d6569dcae5d3dbfe840e18e0e9cb413e9 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 10:36:23 +0000 Subject: [PATCH 134/226] Remove spaces --- src/Common/ProgressIndication.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index a12c025614e8..3776b1d8ae19 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -22,7 +22,7 @@ struct ThreadEventData UInt64 user_ms = 0; UInt64 system_ms = 0; UInt64 memory_usage = 0; - + // -1 used as flag 'is not show for old servers' Int64 peak_memory_usage = -1; }; From e24883d89b680676e047a8eb2d38f746bfd62f54 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 12:24:03 +0000 Subject: [PATCH 135/226] Cut peak memory to fix tests --- .../0_stateless/00416_pocopatch_progress_in_http_headers.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index 3127118c3bae..e77aa1bf250f 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -10,7 +10,7 @@ result="" lines_expected=4 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | sed 's/,\"peak_mem[^}]*//') let counter=counter+1 done echo "$result" @@ -19,7 +19,7 @@ result="" lines_expected=12 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'| sed 's/,\"peak_mem[^}]*//') let counter=counter+1 done echo "$result" From 778efcb95774a3c0c530aff80799feeb054628dc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 17 Jul 2023 11:49:25 +0000 Subject: [PATCH 136/226] Use row_level_filter to restore original header as well --- .../QueryPlan/ReadFromMergeTree.cpp | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index de00f38bf0c4..79d6fd69499e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -141,23 +141,32 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) static void updateSortDescriptionForOutputStream( DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info, PrewhereInfoPtr prewhere_info) { + /// Updating sort description can be done after PREWHERE actions are applied to the header. + /// Aftert PREWHERE actions are applied, column names in header can differ from storage column names due to aliases + /// To mitigate it, we're trying to build original header and use it to deduce sorting description + /// TODO: this approach is fragile, it'd be more robust to update sorting description for the whole plan during plan optimization Block original_header = output_stream.header.cloneEmpty(); - /// build original header - if (prewhere_info && prewhere_info->prewhere_actions) + if (prewhere_info) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + if (prewhere_info->prewhere_actions) + { + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + for (auto & column : original_header) + { + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + column.name = original_node->result_name; + } + } - for (auto & column : original_header) + if (prewhere_info->row_level_filter) { - const auto * original_node = original_column_finder.find(column.name); - if (original_node) + FindOriginalNodeForOutputName original_column_finder(prewhere_info->row_level_filter); + for (auto & column : original_header) { - LOG_DEBUG( - &Poco::Logger::get(__PRETTY_FUNCTION__), - "Found original column '{}' for '{}'", - original_node->result_name, - column.name); - column.name = original_node->result_name; + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + column.name = original_node->result_name; } } } From e8b3310319793250e7c019d690fa3763ff91e8ea Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 17 Jul 2023 14:35:07 -0400 Subject: [PATCH 137/226] touch buffer in advance to avoid page fault --- .../building_and_benchmarking_deflate_qpl.md | 11 ++++++++--- src/Compression/CompressionCodecDeflateQpl.cpp | 7 +++++++ 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 0501c1cbdcb1..baad1f3ea2aa 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -7,11 +7,16 @@ description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec # Build Clickhouse with DEFLATE_QPL -- Make sure your target machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) -- Pass the following flag to CMake when building ClickHouse: +- Make sure your host machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) +- For Clickhouse version [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md) and later, deflate_qpl is enabled by default which means no additional flags required pass to cmake build. +- For Clickhouse version between [v22.8.1.2097-lts](docs/changelogs/v22.8.1.2097-lts.md) and [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md), you need pass the following flag to CMake when building ClickHouse: ``` bash -cmake -DENABLE_QPL=1 .. +cmake -DAVX512=1 -DENABLE_QPL=1 .. +``` +or +``` bash +cmake -DAVX2=1 -DENABLE_QPL=1 .. ``` - For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 859a9c2463ae..41e6a365b6c5 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -405,6 +405,13 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so #if defined(MEMORY_SANITIZER) __msan_unpoison(dest, uncompressed_size); #endif +/// Device IOTLB miss has big perf. impact for IAA accelerators. +/// To avoid page fault, we need touch buffers related to accelerator in advance. + for (char * p = dest; p < dest + uncompressed_size; p += ::getPageSize()/(sizeof(*p))) + { + *p = 0; + } + switch (getDecompressMode()) { case CodecMode::Synchronous: From 47370136e0f3b86a03120d962d8fe43dbcfca9dc Mon Sep 17 00:00:00 2001 From: "pedro.riera" Date: Wed, 14 Jun 2023 10:45:39 +0200 Subject: [PATCH 138/226] new alias for current_database and added current_schemas --- src/Functions/currentDatabase.cpp | 3 +- src/Functions/currentSchemas.cpp | 88 +++++++++++++++++++ .../02788_current_schemas_function.reference | 2 + .../02788_current_schemas_function.sql | 4 + 4 files changed, 96 insertions(+), 1 deletion(-) create mode 100644 src/Functions/currentSchemas.cpp create mode 100644 tests/queries/0_stateless/02788_current_schemas_function.reference create mode 100644 tests/queries/0_stateless/02788_current_schemas_function.sql diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index b1a3cbf58568..b7fd6c4fecc7 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -54,7 +54,8 @@ class FunctionCurrentDatabase : public IFunction REGISTER_FUNCTION(CurrentDatabase) { factory.registerFunction(); - factory.registerAlias("DATABASE", "currentDatabase", FunctionFactory::CaseInsensitive); + factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/currentSchemas.cpp b/src/Functions/currentSchemas.cpp new file mode 100644 index 000000000000..322e719eb170 --- /dev/null +++ b/src/Functions/currentSchemas.cpp @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +class FunctionCurrentSchemas : public IFunction +{ + const String db_name; + +public: + static constexpr auto name = "currentSchemas"; + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getCurrentDatabase()); + } + + explicit FunctionCurrentSchemas(const String & db_name_) : + db_name{db_name_} + { + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + // For compatibility, function implements the same signature as Postgres' + const bool argument_is_valid = arguments.size() == 1 && isBool(arguments.front()); + if (!argument_is_valid) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument for function {} must be bool", getName()); + + return std::make_shared(std::make_shared()); + } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeArray(std::make_shared()) + .createColumnConst(input_rows_count, Array { db_name }); + } +}; + +} + +REGISTER_FUNCTION(CurrentSchema) +{ + factory.registerFunction(FunctionDocumentation + { + .description=R"( +Returns a single-element array with the name of the current database + +Requires a boolean parameter, but it is ignored actually. It is required just for compatibility with the implementation of this function in other DB engines. + +[example:common] +)", + .examples{ + {"common", "SELECT current_schemas(true);", "['default']"} + } + }, + FunctionFactory::CaseInsensitive); + factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::CaseInsensitive); + +} + +} diff --git a/tests/queries/0_stateless/02788_current_schemas_function.reference b/tests/queries/0_stateless/02788_current_schemas_function.reference new file mode 100644 index 000000000000..999c306b3ac7 --- /dev/null +++ b/tests/queries/0_stateless/02788_current_schemas_function.reference @@ -0,0 +1,2 @@ +['default'] +['default'] diff --git a/tests/queries/0_stateless/02788_current_schemas_function.sql b/tests/queries/0_stateless/02788_current_schemas_function.sql new file mode 100644 index 000000000000..408b21c0e342 --- /dev/null +++ b/tests/queries/0_stateless/02788_current_schemas_function.sql @@ -0,0 +1,4 @@ +SELECT current_schemas(true) AS result; +SELECT current_schemas(false) AS result; +SELECT current_schemas(1); -- { serverError 43 } +SELECT current_schemas(); -- { serverError 42 } \ No newline at end of file From 36822543423f7812a3a80b842fa549e32337e45c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 13:49:23 +0000 Subject: [PATCH 139/226] Fix hdfs tests --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index e583d2e30b7c..12fd12d73d57 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -314,7 +314,7 @@ class HDFSSource::URISIterator::Impl : WithContext hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); } /// Skip non-existed files. - while (String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); + while (!hdfs_info && String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); std::optional info; if (hdfs_info) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 0bb840c8d5dd..22b1037ceef3 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -418,7 +418,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext std::function file_progress_callback_) : WithContext(context_) , keys(keys_) - , client(S3::Client::create(client_)) + , client(client_.clone()) , version_id(version_id_) , bucket(bucket_) , request_settings(request_settings_) From 8be1454b28974f7d70f332913eca23e592675fe2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Jul 2023 16:06:03 +0200 Subject: [PATCH 140/226] Fix race --- programs/server/Server.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 91c9a456156e..d1c1a1d200fc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -739,11 +739,12 @@ try [&]() -> std::vector { std::vector metrics; - metrics.reserve(servers_to_start_before_tables.size()); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); + + std::lock_guard lock(servers_lock); for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - std::lock_guard lock(servers_lock); for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; From ea3d9e92f1cd08cfee01832051e304b5f6c59d72 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 14:44:47 +0000 Subject: [PATCH 141/226] Add support of YAML configs for decryption --- src/Common/Config/ConfigProcessor.cpp | 4 +++- src/Common/Config/YAMLParser.cpp | 15 ++++++++++++--- .../test_config_decryption/configs/config.yaml | 11 +++++++++++ tests/integration/test_config_decryption/test.py | 13 +++++++++++-- 4 files changed, 37 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_config_decryption/configs/config.yaml diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 0d40f782be26..faeb56789539 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -28,6 +28,7 @@ #include #if USE_SSL +#include #include #include #include @@ -845,7 +846,8 @@ void ConfigProcessor::savePreprocessedConfig(LoadedConfig & loaded_config, std:: } #if USE_SSL - if (fs::path(preprocessed_path).filename() == "config.xml") + std::string preprocessed_file_name = fs::path(preprocessed_path).filename(); + if (preprocessed_file_name == "config.xml" || preprocessed_file_name == std::format("config{}.xml", PREPROCESSED_SUFFIX)) decryptEncryptedElements(loaded_config); #endif } diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index d96049197260..a1de14afc13e 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -110,9 +110,18 @@ namespace } else { - Poco::AutoPtr xml_key = xml_document->createElement(key); - parent_xml_node.appendChild(xml_key); - processNode(value_node, *xml_key); + if (key == "#text" && value_node.IsScalar()) + { + std::string value = value_node.as(); + Poco::AutoPtr xml_value = xml_document->createTextNode(value); + parent_xml_node.appendChild(xml_value); + } + else + { + Poco::AutoPtr xml_key = xml_document->createElement(key); + parent_xml_node.appendChild(xml_key); + processNode(value_node, *xml_key); + } } } break; diff --git a/tests/integration/test_config_decryption/configs/config.yaml b/tests/integration/test_config_decryption/configs/config.yaml new file mode 100644 index 000000000000..ab4391be3c53 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config.yaml @@ -0,0 +1,11 @@ +encryption_codecs: + aes_128_gcm_siv: + key_hex: 00112233445566778899aabbccddeeff + aes_256_gcm_siv: + key_hex: 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff +max_table_size_to_drop: + '#text': 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + '@encryption_codec': AES_128_GCM_SIV +max_partition_size_to_drop: + '@encryption_codec': AES_256_GCM_SIV + '#text': 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/test.py b/tests/integration/test_config_decryption/test.py index a3cb1bb57f37..dd8cdc2e4e15 100644 --- a/tests/integration/test_config_decryption/test.py +++ b/tests/integration/test_config_decryption/test.py @@ -3,7 +3,8 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/config.xml"]) +node1 = cluster.add_instance("node1", main_configs=["configs/config.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/config.yaml"]) @pytest.fixture(scope="module") @@ -16,7 +17,7 @@ def started_cluster(): cluster.shutdown() -def test_successful_decryption(started_cluster): +def check_node(started_cluster, node): assert ( node.query( "select value from system.server_settings where name ='max_table_size_to_drop'" @@ -29,3 +30,11 @@ def test_successful_decryption(started_cluster): ) == "40000000000\n" ) + + +def test_successful_decryption_xml(started_cluster): + check_node(started_cluster, node1) + + +def test_successful_decryption_yaml(started_cluster): + check_node(started_cluster, node2) From 12df1b2d7f21088e2cdc16e399e97ec0c092e88f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 14:52:38 +0000 Subject: [PATCH 142/226] Fix MD docs style --- docs/en/operations/configuration-files.md | 6 +++--- docs/ru/operations/configuration-files.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 8cd34cc6c303..d1d9fa542ab1 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -67,9 +67,9 @@ Substitutions can also be performed from ZooKeeper. To do this, specify the attr ## Encrypting Configuration {#encryption} -You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec`` with the name of the encryption codec as value to the element to encrypt. +You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec` with the name of the encryption codec as value to the element to encrypt. -Unlike attributes `from_zk`, `from_env` and `incl`` (or element `include``), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. +Unlike attributes `from_zk`, `from_env` and `incl` (or element `include`), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. Example: @@ -82,7 +82,7 @@ Example: admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 91877bdef952..01a91bd41c64 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -87,9 +87,9 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## Шифрование {#encryption} -Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec`` с именем кодека шифрования как значение к элементу, который надо зашифровать. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec` с именем кодека шифрования как значение к элементу, который надо зашифровать. -В отличии от аттрибутов `from_zk`, `from_env` и `incl`` (или элемента `include``), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. +В отличии от аттрибутов `from_zk`, `from_env` и `incl` (или элемента `include`), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. Пример: @@ -102,7 +102,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` From 91b1dd6f7881b2a6291f9426cdfb257191566eda Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 18 Jul 2023 11:00:03 -0400 Subject: [PATCH 143/226] define a function call for zero filling --- src/Compression/CompressionCodecDeflateQpl.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 41e6a365b6c5..5dcd6008b517 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -398,6 +398,14 @@ UInt32 CompressionCodecDeflateQpl::doCompressData(const char * source, UInt32 so return res; } +inline void touchBufferWithZeroFilling(char * buffer, UInt32 buffer_size) +{ + for (char * p = buffer; p < buffer + buffer_size; p += ::getPageSize()/(sizeof(*p))) + { + *p = 0; + } +} + void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { /// QPL library is using AVX-512 with some shuffle operations. @@ -407,10 +415,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so #endif /// Device IOTLB miss has big perf. impact for IAA accelerators. /// To avoid page fault, we need touch buffers related to accelerator in advance. - for (char * p = dest; p < dest + uncompressed_size; p += ::getPageSize()/(sizeof(*p))) - { - *p = 0; - } + touchBufferWithZeroFilling(dest, uncompressed_size); switch (getDecompressMode()) { From 59570b7e2066442f508ad1ff37824f1925b9d3e9 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:07:02 +0000 Subject: [PATCH 144/226] Make encryptValue and decryptValue static --- src/Common/Config/ConfigProcessor.cpp | 4 ++-- src/Common/Config/ConfigProcessor.h | 16 ++++++++-------- src/Compression/CompressionCodecEncrypted.h | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index faeb56789539..6529e94a41df 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -184,7 +184,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) #if USE_SSL -std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) const +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -198,7 +198,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const return hex_value; } -std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) const +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0da304bc0d25..5901101a4100 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,14 +92,6 @@ class ConfigProcessor const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); -#if USE_SSL - /// Encrypt text value - std::string encryptValue(const std::string & codec_name, const std::string & value) const; - - /// Decrypt value - std::string decryptValue(const std::string & codec_name, const std::string & value) const; -#endif - /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ void savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir); @@ -114,6 +106,14 @@ class ConfigProcessor /// Is the file named as result of config preprocessing, not as original files. static bool isPreprocessedFile(const std::string & config_path); +#if USE_SSL + /// Encrypt text value + static std::string encryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt value + static std::string decryptValue(const std::string & codec_name, const std::string & value); +#endif + static inline const auto SUBSTITUTION_ATTRS = {"incl", "from_zk", "from_env"}; private: diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index a8faf3d0b6c1..fafcf4af507e 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,7 +18,7 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; -/// Get method for string name. Throw exception for wrong name +/// Get method for string name. Throw exception for wrong name. EncryptionMethod getEncryptionMethod(const std::string & name); /** This codec encrypts and decrypts blocks with AES-128 in From c8c6c3138488f69d59579712aef10f7d51596431 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:33:50 +0000 Subject: [PATCH 145/226] Change Method into method in exceptions --- src/Compression/CompressionCodecEncrypted.cpp | 10 +++++----- .../test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 01bf6281a0e2..fb870ababa3b 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -38,7 +38,7 @@ EncryptionMethod getEncryptionMethod(const std::string & name) else if (name == "AES_256_GCM_SIV") return AES_256_GCM_SIV; else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", name); } namespace @@ -74,7 +74,7 @@ uint8_t getMethodCode(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -114,7 +114,7 @@ UInt64 methodKeySize(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -139,7 +139,7 @@ auto getMethod(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -215,7 +215,7 @@ auto getMethod(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index c01f5050b00a..a180c37870df 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -14,7 +14,7 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): - start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption Method") + start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG") def test_invalid_chars(): From 50734017ae8c4e05cd37c525b53c46e38846c2c8 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:45:43 +0000 Subject: [PATCH 146/226] Fix test style with black --- .../integration/test_config_decryption/test_wrong_settings.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index a180c37870df..e86f7fa9b398 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -14,7 +14,9 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): - start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG") + start_clickhouse( + "configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG" + ) def test_invalid_chars(): From 0502a4c73e11f4377ead87f00a60d6806e15d489 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Jul 2023 19:09:01 +0200 Subject: [PATCH 147/226] Set startup event during shutdown --- .../ReplicatedMergeTreeRestartingThread.cpp | 6 ++++++ .../ReplicatedMergeTreeRestartingThread.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 16 +++++++++++++++- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index e43cc879e93e..79054ef46daa 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -339,9 +339,15 @@ void ReplicatedMergeTreeRestartingThread::shutdown(bool part_of_full_shutdown) /// Stop restarting_thread before stopping other tasks - so that it won't restart them again. need_stop = true; task->deactivate(); + + /// Explicitly set the event, because the restarting thread will not set it again + if (part_of_full_shutdown) + storage.startup_event.set(); + LOG_TRACE(log, "Restarting thread finished"); setReadonly(part_of_full_shutdown); + } void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 01a877a07e5e..02103272a1f5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -25,6 +26,7 @@ class ReplicatedMergeTreeRestartingThread void start(bool schedule = true) { + LOG_TRACE(log, "Starting restating thread, schedule: {}", schedule); if (schedule) task->activateAndSchedule(); else diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 861f981f7606..53fac578fca6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -186,6 +186,7 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; extern const int NOT_INITIALIZED; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int TABLE_IS_DROPPED; } namespace ActionLocks @@ -3918,7 +3919,10 @@ void StorageReplicatedMergeTree::startBeingLeader() void StorageReplicatedMergeTree::stopBeingLeader() { if (!is_leader) + { + LOG_TRACE(log, "stopBeingLeader called but we are not a leader already"); return; + } LOG_INFO(log, "Stopped being leader"); is_leader = false; @@ -4774,6 +4778,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::fetchExistsPart( void StorageReplicatedMergeTree::startup() { + LOG_TRACE(log, "Starting up table"); startOutdatedDataPartsLoadingTask(); if (attach_thread) { @@ -4795,6 +4800,8 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) since_metadata_err_incr_readonly_metric = true; CurrentMetrics::add(CurrentMetrics::ReadonlyReplica); } + + LOG_TRACE(log, "No connection to ZooKeeper or no metadata in ZooKeeper, will not startup"); return; } @@ -4829,6 +4836,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) if (from_attach_thread) { + LOG_TRACE(log, "Trying to startup table from right now"); /// Try activating replica in current thread. restarting_thread.run(); } @@ -4838,9 +4846,13 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) /// NOTE It does not mean that replication is actually started after receiving this event. /// It only means that an attempt to startup replication was made. /// Table may be still in readonly mode if this attempt failed for any reason. - startup_event.wait(); + while (!startup_event.tryWait(10 * 1000)) + LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } + if (shutdown_prepared_called.load() || shutdown_called.load()) + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() { @@ -4898,6 +4910,8 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); + /// Explicetly set the event, because the restarting thread will not set it again + startup_event.set(); shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } From 8ed2feeb043210070fbd1d3a89e07f7eaa4e9073 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 18 Jul 2023 14:04:31 -0400 Subject: [PATCH 148/226] modify docs for build clickhouse with qpl --- .../building_and_benchmarking_deflate_qpl.md | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index baad1f3ea2aa..4e01b41ab3c1 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -8,16 +8,7 @@ description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec # Build Clickhouse with DEFLATE_QPL - Make sure your host machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) -- For Clickhouse version [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md) and later, deflate_qpl is enabled by default which means no additional flags required pass to cmake build. -- For Clickhouse version between [v22.8.1.2097-lts](docs/changelogs/v22.8.1.2097-lts.md) and [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md), you need pass the following flag to CMake when building ClickHouse: - -``` bash -cmake -DAVX512=1 -DENABLE_QPL=1 .. -``` -or -``` bash -cmake -DAVX2=1 -DENABLE_QPL=1 .. -``` +- deflate_qpl is enabled by default during cmake build. In case you accidentally change it, please double-check build flag: ENABLE_QPL=1 - For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) From b2164ab97eb008e88974e0e06674eeb796b447dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 22:13:28 +0200 Subject: [PATCH 149/226] Fix integration tests --- .../test_concurrent_ttl_merges/configs/users.xml | 7 +++++++ tests/integration/test_concurrent_ttl_merges/test.py | 4 ++-- .../test_replicated_mutations/configs/users.xml | 7 +++++++ tests/integration/test_replicated_mutations/test.py | 4 ++++ .../configs/users.xml | 7 +++++++ .../test_shutdown_wait_unfinished_queries/test.py | 4 ++-- tests/integration/test_storage_kafka/configs/users.xml | 1 + 7 files changed, 30 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_concurrent_ttl_merges/configs/users.xml create mode 100644 tests/integration/test_replicated_mutations/configs/users.xml create mode 100644 tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml diff --git a/tests/integration/test_concurrent_ttl_merges/configs/users.xml b/tests/integration/test_concurrent_ttl_merges/configs/users.xml new file mode 100644 index 000000000000..b0990ca3a60e --- /dev/null +++ b/tests/integration/test_concurrent_ttl_merges/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index 07e91dcbc9f2..f6ba3834c927 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -7,10 +7,10 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/fast_background_pool.xml"], with_zookeeper=True + "node1", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True ) node2 = cluster.add_instance( - "node2", main_configs=["configs/fast_background_pool.xml"], with_zookeeper=True + "node2", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True ) diff --git a/tests/integration/test_replicated_mutations/configs/users.xml b/tests/integration/test_replicated_mutations/configs/users.xml new file mode 100644 index 000000000000..b0990ca3a60e --- /dev/null +++ b/tests/integration/test_replicated_mutations/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_replicated_mutations/test.py b/tests/integration/test_replicated_mutations/test.py index 7479f082b06f..e20bcf367e32 100644 --- a/tests/integration/test_replicated_mutations/test.py +++ b/tests/integration/test_replicated_mutations/test.py @@ -15,6 +15,7 @@ "node2", macros={"cluster": "test1"}, main_configs=["configs/merge_tree.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) @@ -22,12 +23,14 @@ "node3", macros={"cluster": "test2"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) node4 = cluster.add_instance( "node4", macros={"cluster": "test2"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) @@ -35,6 +38,7 @@ "node5", macros={"cluster": "test3"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], ) all_nodes = [node1, node2, node3, node4, node5] diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml b/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml new file mode 100644 index 000000000000..b0990ca3a60e --- /dev/null +++ b/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/test.py b/tests/integration/test_shutdown_wait_unfinished_queries/test.py index ae0710149de6..71f8b9a759d2 100644 --- a/tests/integration/test_shutdown_wait_unfinished_queries/test.py +++ b/tests/integration/test_shutdown_wait_unfinished_queries/test.py @@ -6,10 +6,10 @@ cluster = ClickHouseCluster(__file__) node_wait_queries = cluster.add_instance( - "node_wait_queries", main_configs=["configs/config_wait.xml"], stay_alive=True + "node_wait_queries", main_configs=["configs/config_wait.xml"], user_configs=["configs/users.xml"], stay_alive=True ) node_kill_queries = cluster.add_instance( - "node_kill_queries", main_configs=["configs/config_kill.xml"], stay_alive=True + "node_kill_queries", main_configs=["configs/config_kill.xml"], user_configs=["configs/users.xml"], stay_alive=True ) global result diff --git a/tests/integration/test_storage_kafka/configs/users.xml b/tests/integration/test_storage_kafka/configs/users.xml index 3168de649f86..d13651d5f92f 100644 --- a/tests/integration/test_storage_kafka/configs/users.xml +++ b/tests/integration/test_storage_kafka/configs/users.xml @@ -4,6 +4,7 @@ 1 0 + 0 From ff6e5ff1c547494ed7c6320c5d62bf789d433ae2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jul 2023 20:23:55 +0000 Subject: [PATCH 150/226] Automatic style fix --- tests/integration/test_concurrent_ttl_merges/test.py | 10 ++++++++-- .../test_shutdown_wait_unfinished_queries/test.py | 10 ++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index f6ba3834c927..96264e53522c 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -7,10 +7,16 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True + "node1", + main_configs=["configs/fast_background_pool.xml"], + user_configs=["configs/users.xml"], + with_zookeeper=True, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True + "node2", + main_configs=["configs/fast_background_pool.xml"], + user_configs=["configs/users.xml"], + with_zookeeper=True, ) diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/test.py b/tests/integration/test_shutdown_wait_unfinished_queries/test.py index 71f8b9a759d2..074667fc92f8 100644 --- a/tests/integration/test_shutdown_wait_unfinished_queries/test.py +++ b/tests/integration/test_shutdown_wait_unfinished_queries/test.py @@ -6,10 +6,16 @@ cluster = ClickHouseCluster(__file__) node_wait_queries = cluster.add_instance( - "node_wait_queries", main_configs=["configs/config_wait.xml"], user_configs=["configs/users.xml"], stay_alive=True + "node_wait_queries", + main_configs=["configs/config_wait.xml"], + user_configs=["configs/users.xml"], + stay_alive=True, ) node_kill_queries = cluster.add_instance( - "node_kill_queries", main_configs=["configs/config_kill.xml"], user_configs=["configs/users.xml"], stay_alive=True + "node_kill_queries", + main_configs=["configs/config_kill.xml"], + user_configs=["configs/users.xml"], + stay_alive=True, ) global result From 2d46052d62bf8b7efd16aeb769e278b9df54971b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:35:46 +0000 Subject: [PATCH 151/226] Update description of events "QueryCacheHits/Misses" --- src/Common/ProfileEvents.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0838e0366df1..75d1e493873d 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -57,8 +57,8 @@ M(TableFunctionExecute, "Number of table function calls.") \ M(MarkCacheHits, "Number of times an entry has been found in the mark cache, so we didn't have to load a mark file.") \ M(MarkCacheMisses, "Number of times an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.") \ - M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided).") \ - M(QueryCacheMisses, "Number of times a query result has not been found in the query cache (and required query computation).") \ + M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided). Only updated for SELECT queries with SETTING use_query_cache = 1.") \ + M(QueryCacheMisses, "Number of times a query result has not been found in the query cache (and required query computation). Only updated for SELECT queries with SETTING use_query_cache = 1.") \ M(CreatedReadBufferOrdinary, "Number of times ordinary read buffer was created for reading data (while choosing among other read methods).") \ M(CreatedReadBufferDirectIO, "Number of times a read buffer with O_DIRECT was created for reading data (while choosing among other read methods).") \ M(CreatedReadBufferDirectIOFailed, "Number of times a read buffer with O_DIRECT was attempted to be created for reading data (while choosing among other read methods), but the OS did not allow it (due to lack of filesystem support or other reasons) and we fallen back to the ordinary reading method.") \ From 08409059cc198873ffbf11060bfdabaa0c74f07f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 19 Jul 2023 18:46:20 +0800 Subject: [PATCH 152/226] support alias for new analyzer --- src/Analyzer/Passes/UniqToCountPass.cpp | 108 +++++++++++++----- .../test_rewrite_uniq_to_count/test.py | 16 +-- 2 files changed, 90 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp index ae7952051e72..7533a99107b2 100644 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -21,36 +21,82 @@ bool matchFnUniq(String func_name) || name == "uniqCombined64"; } -bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs) +/// Extract the corresponding projection columns for group by node list. +/// For example: +/// SELECT a as aa, any(b) FROM table group by a; -> aa(ColumnNode) +NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) { - auto * lhs_node = lhs->as(); - auto * rhs_node = rhs->as(); + if (!query_node->hasGroupBy()) + return {}; - if (lhs_node && rhs_node && lhs_node->getColumn() == rhs_node->getColumn()) - return true; - return false; + NamesAndTypes result; + for (const auto & group_by_ele : query_node->getGroupByNode()->getChildren()) + { + const auto & projection_columns = query_node->getProjectionColumns(); + const auto & projection_nodes = query_node->getProjection().getNodes(); + + assert(projection_columns.size() == projection_nodes.size()); + + for (size_t i = 0; i < projection_columns.size(); i++) + { + if (projection_nodes[i]->isEqual(*group_by_ele)) + result.push_back(projection_columns[i]); + } + } + return result; } -bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +/// Whether query_columns equals subquery_columns. +/// query_columns: query columns from query +/// subquery_columns: projection columns from subquery +bool nodeListEquals(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) { - if (lhs.size() != rhs.size()) + if (query_columns.size() != subquery_columns.size()) return false; - for (size_t i = 0; i < lhs.size(); i++) + + for (const auto & query_column : query_columns) { - if (!nodeEquals(lhs[i], rhs[i])) + auto find = std::find_if( + subquery_columns.begin(), + subquery_columns.end(), + [&](const auto & subquery_column) -> bool + { + if (auto * column_node = query_column->as()) + { + return subquery_column == column_node->getColumn(); + } + return false; + }); + + if (find == subquery_columns.end()) return false; } return true; } -bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +/// Whether subquery_columns contains all columns in subquery_columns. +/// query_columns: query columns from query +/// subquery_columns: projection columns from subquery +bool nodeListContainsAll(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) { - if (lhs.size() < rhs.size()) + if (query_columns.size() > subquery_columns.size()) return false; - for (const auto & re : rhs) + + for (const auto & query_column : query_columns) { - auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re); }; - if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) + auto find = std::find_if( + subquery_columns.begin(), + subquery_columns.end(), + [&](const auto & subquery_column) -> bool + { + if (auto * column_node = query_column->as()) + { + return subquery_column == column_node->getColumn(); + } + return false; + }); + + if (find == subquery_columns.end()) return false; } return true; @@ -58,17 +104,14 @@ bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) } -class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContext +class UniqToCountVisitor : public InDepthQueryTreeVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; + using Base = InDepthQueryTreeVisitor; using Base::Base; void visitImpl(QueryTreeNodePtr & node) { - if (!getSettings().optimize_uniq_to_count) - return; - auto * query_node = node->as(); if (!query_node) return; @@ -100,9 +143,11 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContextisDistinct()) return false; - /// uniq expression list == subquery group by expression list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes())) + + /// uniq expression list == subquery projection columns + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjectionColumns())) return false; + return true; }; @@ -111,12 +156,17 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContexthasGroupBy()) return false; + /// uniq argument node list == subquery group by node list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren())) + auto group_by_columns = extractProjectionColumnsForGroupBy(subquery_node); + + if (!nodeListEquals(uniq_arguments_nodes, group_by_columns)) return false; - /// subquery select node list must contain all columns in uniq argument node list - if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes)) + + /// subquery projection columns must contain all columns in uniq argument node list + if (!nodeListContainsAll(uniq_arguments_nodes, subquery_node->getProjectionColumns())) return false; + return true; }; @@ -125,8 +175,11 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContextresolveAsAggregateFunction(std::move(aggregate_function)); function_node->getArguments().getNodes().clear(); + + /// Update projection columns query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); } } @@ -135,7 +188,10 @@ class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContextgetSettings().optimize_uniq_to_count) + return; + + UniqToCountVisitor visitor; visitor.visit(query_tree_node); } diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index d7fa9f394412..e38e57f5cee9 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -83,13 +83,13 @@ def test_rewrite_distinct(started_cluster): ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + check( + "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3, ) # test select expression alias - check_by_old_analyzer( + check( "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3, ) @@ -109,19 +109,19 @@ def test_rewrite_group_by(started_cluster): ) # test select expression alias - check_by_old_analyzer( + check( "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3, ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(t.a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + check( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3, ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + check( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3, ) From b34655e74310dba07c25b06cac817168b6012907 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 19 Jul 2023 17:05:03 +0200 Subject: [PATCH 153/226] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 53fac578fca6..04799a08e37b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4114,7 +4114,7 @@ std::set StorageReplicatedMergeTree::findReplicaUniqueParts(c } if (!our_parts.empty() && our_unique_parts.empty()) - LOG_TRACE(log_, "All parts found on replica"); + LOG_TRACE(log_, "All parts found on replicas"); return our_unique_parts; } From 544081163d751a62dcdfc21e5841c9cb53877cb0 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 19 Jul 2023 17:21:01 +0200 Subject: [PATCH 154/226] Remove redundant deactivate --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 04799a08e37b..06f5330f6d98 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4903,7 +4903,6 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); - mutations_finalizing_task->deactivate(); stopBeingLeader(); if (attach_thread) From dbdac5d823d431fb34405649f7125e76c88f1f05 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 19 Jul 2023 19:34:49 +0000 Subject: [PATCH 155/226] Add query with UNION --- .../02500_remove_redundant_distinct.reference | 29 +++++++++++++++++++ .../02500_remove_redundant_distinct.sh | 12 ++++++++ ...move_redundant_distinct_analyzer.reference | 29 +++++++++++++++++++ 3 files changed, 70 insertions(+) diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference index 2e049dbc9366..763a7cc42866 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference @@ -477,3 +477,32 @@ Expression (Projection) ReadFromStorage (SystemNumbers) -- execute 1 +-- UNION ALL with DISTINCT => do _not_ remove DISTINCT +-- query +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +) +-- explain +Expression (Projection) + Distinct + Distinct (Preliminary DISTINCT) + Union + Expression ((Before ORDER BY + Projection)) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + ReadFromStorage (SystemNumbers) + Expression (( + Projection)) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + ReadFromStorage (SystemNumbers) +-- execute +0 +1 diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh index 41744cc59f96..f07cdca4b5aa 100755 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh @@ -264,3 +264,15 @@ run_query "$query" echo "-- DISTINCT COUNT() with GROUP BY => do _not_ remove DISTINCT" query="select distinct count() from numbers(10) group by number" run_query "$query" + +echo "-- UNION ALL with DISTINCT => do _not_ remove DISTINCT" +query="SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +)" +run_query "$query" diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index c9301c1f0a34..50ca5981cf12 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -479,3 +479,32 @@ Expression (Project names) ReadFromStorage (SystemNumbers) -- execute 1 +-- UNION ALL with DISTINCT => do _not_ remove DISTINCT +-- query +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +) +-- explain +Expression (Project names) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Union + Expression ((Projection + (Change column names to column identifiers + Project names))) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Expression ((Projection + Change column names to column identifiers)) + ReadFromStorage (SystemNumbers) + Expression (( + ( + Project names))) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Expression ((Projection + Change column names to column identifiers)) + ReadFromStorage (SystemNumbers) +-- execute +0 +1 From 2b8e4ebd4c3df56c2d3e445321cedb157c7956f7 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 19:48:39 +0000 Subject: [PATCH 156/226] Allow to disable decoding/encoding path in uri in URL engine --- base/poco/Foundation/include/Poco/URI.h | 6 +++- base/poco/Foundation/src/URI.cpp | 39 ++++++++++++++++++------- docs/en/operations/settings/settings.md | 6 ++++ src/Core/Settings.h | 1 + src/IO/ReadWriteBufferFromHTTP.cpp | 6 ++-- src/Storages/StorageURL.cpp | 2 +- 6 files changed, 45 insertions(+), 15 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index 1880af4ccd2d..5e6e7efd9383 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ class Foundation_API URI URI(); /// Creates an empty URI. - explicit URI(const std::string & uri); + explicit URI(const std::string & uri, bool decode_and_encode_path = true); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -350,6 +350,8 @@ class Foundation_API URI static const std::string ILLEGAL; private: + void encodePath(std::string & encodedStr) const; + std::string _scheme; std::string _userInfo; std::string _host; @@ -357,6 +359,8 @@ class Foundation_API URI std::string _path; std::string _query; std::string _fragment; + + bool _decode_and_encode_path = true; }; diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 5543e02b2791..91a82868dcf7 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -36,8 +36,8 @@ URI::URI(): } -URI::URI(const std::string& uri): - _port(0) +URI::URI(const std::string& uri, bool decode_and_encode_path): + _port(0), _decode_and_encode_path(decode_and_encode_path) { parse(uri); } @@ -107,7 +107,8 @@ URI::URI(const URI& uri): _port(uri._port), _path(uri._path), _query(uri._query), - _fragment(uri._fragment) + _fragment(uri._fragment), + _decode_and_encode_path(uri._decode_and_encode_path) { } @@ -119,7 +120,8 @@ URI::URI(const URI& baseURI, const std::string& relativeURI): _port(baseURI._port), _path(baseURI._path), _query(baseURI._query), - _fragment(baseURI._fragment) + _fragment(baseURI._fragment), + _decode_and_encode_path(baseURI._decode_and_encode_path) { resolve(relativeURI); } @@ -151,6 +153,7 @@ URI& URI::operator = (const URI& uri) _path = uri._path; _query = uri._query; _fragment = uri._fragment; + _decode_and_encode_path = uri._decode_and_encode_path; } return *this; } @@ -181,6 +184,7 @@ void URI::swap(URI& uri) std::swap(_path, uri._path); std::swap(_query, uri._query); std::swap(_fragment, uri._fragment); + std::swap(_decode_and_encode_path, uri._decode_and_encode_path); } @@ -201,7 +205,7 @@ std::string URI::toString() const std::string uri; if (isRelative()) { - encode(_path, RESERVED_PATH, uri); + encodePath(uri); } else { @@ -217,7 +221,7 @@ std::string URI::toString() const { if (!auth.empty() && _path[0] != '/') uri += '/'; - encode(_path, RESERVED_PATH, uri); + encodePath(uri); } else if (!_query.empty() || !_fragment.empty()) { @@ -313,7 +317,10 @@ void URI::setAuthority(const std::string& authority) void URI::setPath(const std::string& path) { _path.clear(); - decode(path, _path); + if (_decode_and_encode_path) + decode(path, _path); + else + _path = path; } @@ -418,7 +425,7 @@ void URI::setPathEtc(const std::string& pathEtc) std::string URI::getPathEtc() const { std::string pathEtc; - encode(_path, RESERVED_PATH, pathEtc); + encodePath(pathEtc); if (!_query.empty()) { pathEtc += '?'; @@ -436,7 +443,7 @@ std::string URI::getPathEtc() const std::string URI::getPathAndQuery() const { std::string pathAndQuery; - encode(_path, RESERVED_PATH, pathAndQuery); + encodePath(pathAndQuery); if (!_query.empty()) { pathAndQuery += '?'; @@ -626,6 +633,8 @@ void URI::encode(const std::string& str, const std::string& reserved, std::strin for (std::string::const_iterator it = str.begin(); it != str.end(); ++it) { char c = *it; + if (c == '%') + throw std::runtime_error("WTF"); if ((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || @@ -681,6 +690,13 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa } } +void URI::encodePath(std::string & encodedStr) const +{ + if (_decode_and_encode_path) + encode(_path, RESERVED_PATH, encodedStr); + else + encodedStr = _path; +} bool URI::isWellKnownPort() const { @@ -820,7 +836,10 @@ void URI::parsePath(std::string::const_iterator& it, const std::string::const_it { std::string path; while (it != end && *it != '?' && *it != '#') path += *it++; - decode(path, _path); + if (_decode_and_encode_path) + decode(path, _path); + else + _path = path; } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d0..db5d1a2f5d9c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3466,6 +3466,12 @@ Possible values: Default value: `0`. +## decode_and_encode_path_in_url {#decode_and_encode_path_in_url} + +Enables or disables decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. + +Enabled by default. + ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Adds a modifier `SYNC` to all `DROP` and `DETACH` queries. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6c..ffa72d841bed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -621,6 +621,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ + M(Bool, decode_and_encode_path_in_url, true, "Enables or disables decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 6d1c0f7aafa1..eea801ce65ed 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -305,12 +305,12 @@ void ReadWriteBufferFromHTTPBase::callWithRedirects(Poco::N current_session = session; call(current_session, response, method_, throw_on_all_errors, for_object_info); - Poco::URI prev_uri = uri; + saved_uri_redirect = uri; while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect = getUriAfterRedirect(prev_uri, response); - prev_uri = uri_redirect; + Poco::URI uri_redirect = getUriAfterRedirect(*saved_uri_redirect, response); + saved_uri_redirect = uri_redirect; if (remote_host_filter) remote_host_filter->checkURL(uri_redirect); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index e6953afe68e4..4cfefbc55273 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -389,7 +389,7 @@ std::pair> StorageURLSource: for (; option != end; ++option) { bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option); + auto request_uri = Poco::URI(*option, context->getSettingsRef().decode_and_encode_path_in_url); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); From 483ddb53ebfa01c02deda76a39bc44cc08df4f00 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 19:51:58 +0000 Subject: [PATCH 157/226] Fixes --- base/poco/Foundation/src/URI.cpp | 2 -- docs/en/engines/table-engines/special/url.md | 1 + docs/en/sql-reference/table-functions/url.md | 3 ++- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 91a82868dcf7..9bad1b39a871 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -633,8 +633,6 @@ void URI::encode(const std::string& str, const std::string& reserved, std::strin for (std::string::const_iterator it = str.begin(); it != str.end(); ++it) { char c = *it; - if (c == '%') - throw std::runtime_error("WTF"); if ((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 26d4975954fa..9f2bf177c963 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -106,3 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 2ab43f1b895e..96f36f039496 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -56,7 +56,8 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. -**See Also** +- **See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) From cbcd48979cab1a3a4a0f0d5baaf8077164887cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Jul 2023 13:04:43 +0200 Subject: [PATCH 158/226] Fix race one more time --- programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d1c1a1d200fc..774c3f223a6f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -739,9 +739,10 @@ try [&]() -> std::vector { std::vector metrics; - metrics.reserve(servers_to_start_before_tables.size() + servers.size()); std::lock_guard lock(servers_lock); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); + for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); From fe934d3059936cd203952cfe5881ff7243001ae9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:38:41 +0000 Subject: [PATCH 159/226] Make better --- docs/en/engines/table-engines/special/url.md | 2 +- docs/en/operations/settings/settings.md | 6 +++--- docs/en/sql-reference/table-functions/url.md | 4 ++-- src/Core/Settings.h | 2 +- src/Storages/StorageURL.cpp | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 9f2bf177c963..f556df0a0887 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -106,4 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. +- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) -allows to disable decoding/encoding path in uri. Disabled by default. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index db5d1a2f5d9c..d138b07d3ae2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3466,11 +3466,11 @@ Possible values: Default value: `0`. -## decode_and_encode_path_in_url {#decode_and_encode_path_in_url} +## disable_url_encoding {#disable_url_encoding} -Enables or disables decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. +Allows to disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. -Enabled by default. +Disabled by default. ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 96f36f039496..677ed0119608 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -56,8 +56,8 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. +- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) - allows to disable decoding/encoding path in uri. Disabled by default. -- **See Also** +**See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ffa72d841bed..5dc404941157 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -621,7 +621,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ - M(Bool, decode_and_encode_path_in_url, true, "Enables or disables decoding/encoding path in uri in URL table engine", 0) \ + M(Bool, disable_url_encoding, false, " Allows to disable decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4cfefbc55273..0c915f54cff5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -389,7 +389,7 @@ std::pair> StorageURLSource: for (; option != end; ++option) { bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option, context->getSettingsRef().decode_and_encode_path_in_url); + auto request_uri = Poco::URI(*option, context->getSettingsRef().disable_url_encoding); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); From f6a44f8eedce98bd50ceee72e5fdc4da1a82a43a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:40:41 +0000 Subject: [PATCH 160/226] Better --- base/poco/Foundation/include/Poco/URI.h | 6 +++-- base/poco/Foundation/src/URI.cpp | 32 +++++++++++++------------ 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index 5e6e7efd9383..f4505147cedd 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ class Foundation_API URI URI(); /// Creates an empty URI. - explicit URI(const std::string & uri, bool decode_and_encode_path = true); + explicit URI(const std::string & uri, bool disable_url_encoding = true); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -351,6 +351,8 @@ class Foundation_API URI private: void encodePath(std::string & encodedStr) const; + void decodePath(const std::string & encodedStr); + std::string _scheme; std::string _userInfo; @@ -360,7 +362,7 @@ class Foundation_API URI std::string _query; std::string _fragment; - bool _decode_and_encode_path = true; + bool _disable_url_encoding = true; }; diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 9bad1b39a871..3354c69d1883 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -37,7 +37,7 @@ URI::URI(): URI::URI(const std::string& uri, bool decode_and_encode_path): - _port(0), _decode_and_encode_path(decode_and_encode_path) + _port(0), _disable_url_encoding(decode_and_encode_path) { parse(uri); } @@ -108,7 +108,7 @@ URI::URI(const URI& uri): _path(uri._path), _query(uri._query), _fragment(uri._fragment), - _decode_and_encode_path(uri._decode_and_encode_path) + _disable_url_encoding(uri._disable_url_encoding) { } @@ -121,7 +121,7 @@ URI::URI(const URI& baseURI, const std::string& relativeURI): _path(baseURI._path), _query(baseURI._query), _fragment(baseURI._fragment), - _decode_and_encode_path(baseURI._decode_and_encode_path) + _disable_url_encoding(baseURI._disable_url_encoding) { resolve(relativeURI); } @@ -153,7 +153,7 @@ URI& URI::operator = (const URI& uri) _path = uri._path; _query = uri._query; _fragment = uri._fragment; - _decode_and_encode_path = uri._decode_and_encode_path; + _disable_url_encoding = uri._disable_url_encoding; } return *this; } @@ -184,7 +184,7 @@ void URI::swap(URI& uri) std::swap(_path, uri._path); std::swap(_query, uri._query); std::swap(_fragment, uri._fragment); - std::swap(_decode_and_encode_path, uri._decode_and_encode_path); + std::swap(_disable_url_encoding, uri._disable_url_encoding); } @@ -317,10 +317,7 @@ void URI::setAuthority(const std::string& authority) void URI::setPath(const std::string& path) { _path.clear(); - if (_decode_and_encode_path) - decode(path, _path); - else - _path = path; + decodePath(path); } @@ -690,10 +687,18 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa void URI::encodePath(std::string & encodedStr) const { - if (_decode_and_encode_path) + if (_disable_url_encoding) + encodedStr = _path; + else encode(_path, RESERVED_PATH, encodedStr); +} + +void URI::decodePath(const std::string & encodedStr) +{ + if (_disable_url_encoding) + _path = encodedStr; else - encodedStr = _path; + decode(encodedStr, _path); } bool URI::isWellKnownPort() const @@ -834,10 +839,7 @@ void URI::parsePath(std::string::const_iterator& it, const std::string::const_it { std::string path; while (it != end && *it != '?' && *it != '#') path += *it++; - if (_decode_and_encode_path) - decode(path, _path); - else - _path = path; + decodePath(path); } From 8649c84461f3c27bdf9fcab4db1884b21603dc2e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 13:28:37 +0000 Subject: [PATCH 161/226] Remove conditional linking --- utils/config-processor/CMakeLists.txt | 6 +----- utils/keeper-bench/CMakeLists.txt | 6 +----- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 4394083a1c36..80c3535ef4eb 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,6 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -if (ENABLE_SSL) - target_link_libraries(config-processor PRIVATE dbms) -else () - target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) -endif () +target_link_libraries(config-processor PRIVATE dbms) diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index e8daec9e1642..5514c34f4efb 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -4,9 +4,5 @@ if (NOT TARGET ch_contrib::rapidjson) endif () clickhouse_add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -if (ENABLE_SSL) - target_link_libraries(keeper-bench PRIVATE dbms) -else () - target_link_libraries(keeper-bench PRIVATE clickhouse_common_config_no_zookeeper_log) -endif () +target_link_libraries(keeper-bench PRIVATE dbms) target_link_libraries(keeper-bench PRIVATE ch_contrib::rapidjson) From 8adf57a6981610936acc84f3c69342682952ff0a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 14:18:32 +0000 Subject: [PATCH 162/226] Fix text in comments and improve exception handling --- src/Common/examples/encrypt_decrypt.cpp | 2 +- tests/integration/test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 542e173deb92..2d8c5a5f61f2 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -3,7 +3,7 @@ #include #include -/** This test program encrypts or decrypts text values using AES_128_GCM_SIV or AES_256_GCM_SIV codecs. +/** This test program encrypts or decrypts text values using a symmetric encryption codec like AES_128_GCM_SIV or AES_256_GCM_SIV. * Keys for codecs are loaded from section of configuration file. * * How to use: diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index e86f7fa9b398..e0fbd4b29482 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -10,7 +10,7 @@ def start_clickhouse(config, err_msg): cluster.start() except Exception as e: caught_exception = str(e) - assert caught_exception.find(err_msg) != -1 + assert err_msg in caught_exception def test_wrong_method(): From 714a3a8d121326e2d908648bfc0e76e09f6e0815 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 12:23:22 +0200 Subject: [PATCH 163/226] Don't do it on drop --- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 06f5330f6d98..841b646a1264 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3994,6 +3994,13 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(StorageReplicatedMergeTree::ShutdownDeadline shutdown_deadline_) { + /// Will be true in case in case of query + if (CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr) + { + LOG_TRACE(log, "Will not wait for unique parts to be fetched by other replicas because shutdown called from DROP/DETACH query"); + return; + } + if (!shutdown_called.load()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); @@ -4951,7 +4958,6 @@ void StorageReplicatedMergeTree::shutdown() flushAndPrepareForShutdown(); - auto settings_ptr = getSettings(); if (!shutdown_deadline.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); @@ -6311,7 +6317,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { - bool stop_waiting_itself = waiting_itself && partial_shutdown_called; + bool stop_waiting_itself = waiting_itself && (partial_shutdown_called || shutdown_prepared_called || shutdown_called); bool timeout_exceeded = check_timeout && wait_for_inactive_timeout < time_waiting.elapsedSeconds(); bool stop_waiting_inactive = (!wait_for_inactive || timeout_exceeded) && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"); From 3798bd6f509b7fc0591201c66c2e0d1b254835dd Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 12:52:07 +0000 Subject: [PATCH 164/226] Replace test by text_to_encrypt --- src/Common/examples/encrypt_decrypt.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 2d8c5a5f61f2..503802016cbb 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -7,7 +7,7 @@ * Keys for codecs are loaded from section of configuration file. * * How to use: - * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test + * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV text_to_encrypt */ int main(int argc, char ** argv) @@ -22,7 +22,7 @@ int main(int argc, char ** argv) << "action: -e for encryption and -d for decryption." << std::endl << "codec: AES_128_GCM_SIV or AES_256_GCM_SIV." << std::endl << std::endl << "Example:" << std::endl - << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test"; + << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV text_to_encrypt"; return 3; } From 0aed62ec73b8de4614506f5b72a086d8d10db4aa Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:03:25 +0000 Subject: [PATCH 165/226] Add codec name into exception message --- src/Compression/CompressionCodecEncrypted.cpp | 2 +- ..._no_encryption_codecs.xml => config_no_encryption_key.xml} | 1 - .../integration/test_config_decryption/test_wrong_settings.py | 4 ++-- 3 files changed, 3 insertions(+), 4 deletions(-) rename tests/integration/test_config_decryption/configs/{config_no_encryption_codecs.xml => config_no_encryption_key.xml} (52%) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index fb870ababa3b..3f4e35a78a46 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -588,7 +588,7 @@ String CompressionCodecEncrypted::Configuration::getKey(EncryptionMethod method, if (current_params->keys_storage[method].contains(key_id)) key = current_params->keys_storage[method].at(key_id); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config", key_id); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config for {} encryption codec", key_id, getMethodName(method)); return key; } diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml similarity index 52% rename from tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml rename to tests/integration/test_config_decryption/configs/config_no_encryption_key.xml index 07bf69d17c8b..5f7769f74031 100644 --- a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml @@ -1,4 +1,3 @@ 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index e0fbd4b29482..626109645021 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -26,9 +26,9 @@ def test_invalid_chars(): ) -def test_no_encryption_codecs(): +def test_no_encryption_key(): start_clickhouse( - "configs/config_no_encryption_codecs.xml", "There is no key 0 in config" + "configs/config_no_encryption_key.xml", "There is no key 0 in config for AES_128_GCM_SIV encryption codec" ) From 10ec06917506c1a61caadf4c680bd0148520426f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:29:40 +0000 Subject: [PATCH 166/226] Improve exception message text --- src/Common/Config/ConfigProcessor.cpp | 2 +- tests/integration/test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 6529e94a41df..73fc5c58b2f2 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -232,7 +232,7 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { const NodeListPtr children = element.childNodes(); if (children->length() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} cannot contain nested elements", node->nodeName()); Node * text_node = node->firstChild(); if (text_node->nodeType() != Node::TEXT_NODE) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index 626109645021..da32a8f0ac82 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -33,4 +33,4 @@ def test_no_encryption_key(): def test_subnodes(): - start_clickhouse("configs/config_subnodes.xml", "should have only one text node") + start_clickhouse("configs/config_subnodes.xml", "cannot contain nested elements") From 1daa26c74130003a4039dcc809b9d3d0a5bcba95 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:31:42 +0000 Subject: [PATCH 167/226] Fix black formatting --- .../integration/test_config_decryption/test_wrong_settings.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index da32a8f0ac82..b148f9a051aa 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -28,7 +28,8 @@ def test_invalid_chars(): def test_no_encryption_key(): start_clickhouse( - "configs/config_no_encryption_key.xml", "There is no key 0 in config for AES_128_GCM_SIV encryption codec" + "configs/config_no_encryption_key.xml", + "There is no key 0 in config for AES_128_GCM_SIV encryption codec", ) From 5fb5ba71edbaf664045871b0fc8d6d5d6f5f45e6 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 15:40:53 +0000 Subject: [PATCH 168/226] Throw exception when several text nodes found in YAML for element node --- src/Common/Config/YAMLParser.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index a1de14afc13e..72706cb98ba7 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -112,6 +112,11 @@ namespace { if (key == "#text" && value_node.IsScalar()) { + for (Node * child_node = parent_xml_node.firstChild(); child_node; child_node = child_node->nextSibling()) + if (child_node->nodeType() == Node::TEXT_NODE) + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, + "YAMLParser has encountered node with several text nodes " + "and cannot continue parsing of the file"); std::string value = value_node.as(); Poco::AutoPtr xml_value = xml_document->createTextNode(value); parent_xml_node.appendChild(xml_value); From a2b170a18e7db041eb41e631f693b3ddec8e79a7 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 17:42:55 +0200 Subject: [PATCH 169/226] Avoid exception which I didn't understand --- src/Storages/StorageReplicatedMergeTree.cpp | 33 +++++++++++++-------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 841b646a1264..bc8dbfa0e1fc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4905,20 +4905,29 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (shutdown_prepared_called.exchange(true)) return; - auto settings_ptr = getSettings(); - /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. - fetcher.blocker.cancelForever(); - merger_mutator.merges_blocker.cancelForever(); - parts_mover.moves_blocker.cancelForever(); - stopBeingLeader(); + try + { + auto settings_ptr = getSettings(); + /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. + fetcher.blocker.cancelForever(); + merger_mutator.merges_blocker.cancelForever(); + parts_mover.moves_blocker.cancelForever(); + stopBeingLeader(); - if (attach_thread) - attach_thread->shutdown(); + if (attach_thread) + attach_thread->shutdown(); - restarting_thread.shutdown(/* part_of_full_shutdown */true); - /// Explicetly set the event, because the restarting thread will not set it again - startup_event.set(); - shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); + restarting_thread.shutdown(/* part_of_full_shutdown */true); + /// Explicetly set the event, because the restarting thread will not set it again + startup_event.set(); + shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); + } + catch (...) + { + /// Don't wait anything in case of inproper prepare for shutdown + shutdown_deadline.emplace(std::chrono::system_clock::now()); + throw; + } } void StorageReplicatedMergeTree::partialShutdown() From ed97284bfae5b18f8dbc8841e8b296edd45cd286 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Jul 2023 19:28:28 +0200 Subject: [PATCH 170/226] Update src/Storages/StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc8dbfa0e1fc..f191440442d8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4924,7 +4924,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() } catch (...) { - /// Don't wait anything in case of inproper prepare for shutdown + /// Don't wait anything in case of improper prepare for shutdown shutdown_deadline.emplace(std::chrono::system_clock::now()); throw; } From ae8f511ac5ffc6191394dd9fbfed9a0b082102e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:27:07 +0200 Subject: [PATCH 171/226] Fix a test --- tests/integration/test_zero_copy_fetch/configs/users.xml | 7 +++++++ tests/integration/test_zero_copy_fetch/test.py | 2 ++ 2 files changed, 9 insertions(+) create mode 100644 tests/integration/test_zero_copy_fetch/configs/users.xml diff --git a/tests/integration/test_zero_copy_fetch/configs/users.xml b/tests/integration/test_zero_copy_fetch/configs/users.xml new file mode 100644 index 000000000000..b0990ca3a60e --- /dev/null +++ b/tests/integration/test_zero_copy_fetch/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index 4f3d42096c34..dc79e5d8723d 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -19,12 +19,14 @@ def started_cluster(): cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], + user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, ) cluster.add_instance( "node2", main_configs=["configs/storage_conf.xml"], + user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, ) From afdda489bdfb27d1db2a7554223f5dfcb8cca7b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 19:53:16 +0200 Subject: [PATCH 172/226] Fix test --- .../02790_sql_standard_fetch.reference | 72 +++++++++---------- .../0_stateless/02790_sql_standard_fetch.sql | 12 ++-- 2 files changed, 42 insertions(+), 42 deletions(-) diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference index 429eecbc9364..270af6e5c174 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.reference +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -1,36 +1,36 @@ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name──────────┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────────────┴────────────┴────────┘ +┌─id─┬─name──────────┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────────────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql index 4204279a7461..07a806eddf91 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.sql +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -1,33 +1,33 @@ -- https://antonz.org/sql-fetch/ CREATE TEMPORARY TABLE employees (id UInt64, name String, department String, salary UInt64); -INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', '84'); +INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', 84); -- Determinism SET max_threads = 1, parallelize_output_from_storages = 0; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc limit 5 format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows only format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows with ties format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch next 5 rows only format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch first 5 rows only From 7a24de801d93957cd87e8a1d2f726b934912b038 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 19:53:45 +0200 Subject: [PATCH 173/226] Fix test --- .../02790_sql_standard_fetch.reference | 72 +++++++++---------- .../0_stateless/02790_sql_standard_fetch.sql | 10 +-- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference index 270af6e5c174..ef578b526da8 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.reference +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -1,36 +1,36 @@ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name──────────┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────────────┴────────────┴────────┘ -┌─id─┬─name──────────┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name──────────┬─department─┬─salary─┐ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +│ Grace │ it │ 90 │ +│ Emma │ it │ 84 │ +└───────────────┴────────────┴────────┘ +┌─name──────────┬─department─┬─salary─┐ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +│ Grace │ it │ 90 │ +│ Emma │ it │ 84 │ +└───────────────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql index 07a806eddf91..638cc66682d3 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.sql +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -6,28 +6,28 @@ INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), -- Determinism SET max_threads = 1, parallelize_output_from_storages = 0; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc limit 5 format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows only format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows with ties format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch next 5 rows only format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch first 5 rows only From 039cac69cf6d30cc58c8531b1efac4d9847cb599 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 23 Jul 2023 18:35:37 +0200 Subject: [PATCH 174/226] Fix test_insert_same_partition_and_merge by increasing wait time --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 761b5257a348..86b70f8db709 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -215,7 +215,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): if attempt == 59: assert parts_count == "(1)" - time.sleep(1) + time.sleep(10) assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" assert ( From 169b9d5cc0c8dc54d31bc7229204b195f294c877 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:49:06 +0200 Subject: [PATCH 175/226] Fix tidy --- src/Functions/GregorianDate.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index da1172c8916a..aaaeeb7339d5 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -20,12 +20,12 @@ namespace ErrorCodes namespace { - static inline constexpr bool is_leap_year(int32_t year) + inline constexpr bool is_leap_year(int32_t year) { return (year % 4 == 0) && ((year % 400 == 0) || (year % 100 != 0)); } - static inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) + inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) { switch (month) { @@ -49,7 +49,7 @@ namespace /** Integer division truncated toward negative infinity. */ template - static inline constexpr I div(I x, J y) + inline constexpr I div(I x, J y) { const auto y_cast = static_cast(y); if (x > 0 && y_cast < 0) @@ -63,7 +63,7 @@ namespace /** Integer modulus, satisfying div(x, y)*y + mod(x, y) == x. */ template - static inline constexpr I mod(I x, J y) + inline constexpr I mod(I x, J y) { const auto y_cast = static_cast(y); const auto r = x % y_cast; @@ -76,13 +76,13 @@ namespace /** Like std::min(), but the type of operands may differ. */ template - static inline constexpr I min(I x, J y) + inline constexpr I min(I x, J y) { const auto y_cast = static_cast(y); return x < y_cast ? x : y_cast; } - static inline char readDigit(ReadBuffer & in) + inline char readDigit(ReadBuffer & in) { char c; if (!in.read(c)) @@ -93,7 +93,7 @@ namespace return c - '0'; } - static inline bool tryReadDigit(ReadBuffer & in, char & c) + inline bool tryReadDigit(ReadBuffer & in, char & c) { if (in.read(c) && c >= '0' && c <= '9') { From d7f7f16fbcfa8063e295708b4feb3b0079ad05f0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:44:13 +0200 Subject: [PATCH 176/226] Introduce IStorage::supportsTrivialCountOptimization() Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- src/Planner/PlannerJoinTree.cpp | 3 +++ src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/StorageMaterializedMySQL.h | 2 ++ 5 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d07a65215445..fc3ea3a13ca2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2274,8 +2274,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle && !settings.allow_experimental_query_deduplication && !settings.empty_result_for_aggregation_by_empty_set && storage - && storage->getName() != "MaterializedMySQL" - && !storage->hasLightweightDeletedMask() + && storage->supportsTrivialCountOptimization() && query_info.filter_asts.empty() && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5d8f8ca8741f..c118fccded43 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -182,6 +182,9 @@ bool applyTrivialCountIfPossible( return false; const auto & storage = table_node.getStorage(); + if (!storage->supportsTrivialCountOptimization()) + return false; + auto storage_id = storage->getStorageID(); auto row_policy_filter = query_context->getRowPolicyFilter(storage_id.getDatabaseName(), storage_id.getTableName(), diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 76641b656a21..701e02a85ac7 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -254,6 +254,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } + /// Return true if the trivial count query could be optimized without reading the data at all. + virtual bool supportsTrivialCountOptimization() const { return false; } + private: StorageID storage_id; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 41fc46578542..5e6b043c31c7 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -434,6 +434,8 @@ class MergeTreeData : public IStorage, public WithMutableContext bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } + bool supportsTrivialCountOptimization() const override { return !hasLightweightDeletedMask(); } + NamesAndTypesList getVirtuals() const override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 08fbb61960f7..e6fcbc203e6f 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -41,6 +41,8 @@ class StorageMaterializedMySQL final : public StorageProxy void drop() override { nested_storage->drop(); } + bool supportsTrivialCountOptimization() const override { return false; } + private: [[noreturn]] static void throwNotAllowed() { From a0070eda02736903b984518daf3d1c79bfe5fd94 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:48:21 +0200 Subject: [PATCH 177/226] Slightly optimize code in ClusterProxy::executeQuery() Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 3dea52faf461..5efba383e4b9 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -176,11 +176,9 @@ void executeQuery( 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) + ASTPtr query_ast_for_shard = query_ast->clone(); + if (sharding_key_expr && 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, @@ -191,8 +189,6 @@ void executeQuery( OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); visitor.visit(query_ast_for_shard); } - else - query_ast_for_shard = query_ast->clone(); if (shard_filter_generator) { From 67095d2150cafc91c0eebea4a17a8dc5f17b307c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:48:58 +0200 Subject: [PATCH 178/226] Fix comment for function argument in TableFunctionRemote Signed-off-by: Azat Khuzhin --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 4143014a7b32..e6d72ddf17b9 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -264,7 +264,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr secure, /* priority= */ Priority{1}, /* cluster_name= */ "", - /* password= */ "" + /* cluster_secret= */ "" }; cluster = std::make_shared(context->getSettingsRef(), names, params); } From b22247609036020e9bc4da64f1a297e49c29edfa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 14:19:08 +0200 Subject: [PATCH 179/226] Add ability to pass table for connections checks per-shard to ReadFromRemote Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 1 + src/Interpreters/ClusterProxy/SelectStreamFactory.h | 2 ++ src/Processors/QueryPlan/ReadFromRemote.cpp | 6 ++++-- src/Processors/QueryPlan/ReadFromRemote.h | 1 + 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0cf3f3609944..953e38d56cd5 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -124,6 +124,7 @@ void SelectStreamFactory::createForShard( { remote_shards.emplace_back(Shard{ .query = query_ast, + .main_table = main_table, .header = header, .shard_info = shard_info, .lazy = lazy, diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 030c0b77dd52..1cc5a3b1a77e 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -50,6 +50,8 @@ class SelectStreamFactory { /// Query and header may be changed depending on shard. ASTPtr query; + /// Used to check the table existence on remote node + StorageID main_table; Block header; Cluster::ShardInfo shard_info; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 5cc13f45df40..7a99c3632328 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -162,7 +162,9 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (my_table_func_ptr) try_results = my_shard.shard_info.pool->getManyForTableFunction(timeouts, ¤t_settings, PoolMode::GET_MANY); else - try_results = my_shard.shard_info.pool->getManyChecked(timeouts, ¤t_settings, PoolMode::GET_MANY, my_main_table.getQualifiedName()); + try_results = my_shard.shard_info.pool->getManyChecked( + timeouts, ¤t_settings, PoolMode::GET_MANY, + my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName()); } catch (const Exception & ex) { @@ -241,7 +243,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact remote_query_executor->setPoolMode(PoolMode::GET_MANY); if (!table_func_ptr) - remote_query_executor->setMainTable(main_table); + remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); addConvertingActions(pipes.back(), output_stream->header); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index d4005d81f1be..ac869cd89f94 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -22,6 +22,7 @@ using ThrottlerPtr = std::shared_ptr; class ReadFromRemote final : public ISourceStep { public: + /// @param main_table_ if Shards contains main_table then this parameter will be ignored ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, From 83c0f03b98d6b3cbd10f9690256aed2fada47177 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 14:21:53 +0200 Subject: [PATCH 180/226] Change signature of the updateSettingsForCluster() to avoid cluster requirement Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 11 ++++++++--- src/Interpreters/ClusterProxy/executeQuery.h | 8 ++++++-- src/Storages/getStructureOfRemoteTable.cpp | 4 ++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 5efba383e4b9..2fed626ffb7e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -35,7 +35,12 @@ namespace ErrorCodes namespace ClusterProxy { -ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info, Poco::Logger * log) +ContextMutablePtr updateSettingsForCluster(bool interserver_mode, + ContextPtr context, + const Settings & settings, + const StorageID & main_table, + const SelectQueryInfo * query_info, + Poco::Logger * log) { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); @@ -43,7 +48,7 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c /// If "secret" (in remote_servers) is not in use, /// user on the shard is not the same as the user on the initiator, /// hence per-user limits should not be applied. - if (cluster.getSecret().empty()) + if (!interserver_mode) { /// Does not matter on remote servers, because queries are sent under different user. new_settings.max_concurrent_queries_for_user = 0; @@ -170,7 +175,7 @@ void executeQuery( std::vector plans; SelectStreamFactory::Shards remote_shards; - auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log); + auto new_context = updateSettingsForCluster(!query_info.getCluster()->getSecret().empty(), context, settings, main_table, &query_info, log); new_context->increaseDistributedDepth(); size_t shards = query_info.getCluster()->getShardCount(); diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 41f6da55686b..511914e99e47 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -34,8 +34,12 @@ class SelectStreamFactory; /// - optimize_skip_unused_shards_nesting /// /// @return new Context with adjusted settings -ContextMutablePtr updateSettingsForCluster( - const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info = nullptr, Poco::Logger * log = nullptr); +ContextMutablePtr updateSettingsForCluster(bool interserver_mode, + ContextPtr context, + const Settings & settings, + const StorageID & main_table, + const SelectQueryInfo * query_info = nullptr, + Poco::Logger * log = nullptr); using AdditionalShardFilterGenerator = std::function; /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f47..cbed05e30edd 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -58,7 +58,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( } ColumnsDescription res; - auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), table_id); /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), /// since this is a service query and should not lead to query failure. @@ -177,7 +177,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( const auto & shards_info = cluster.getShardsInfo(); auto query = "DESC TABLE " + remote_table_id.getFullTableName(); - auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), remote_table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), remote_table_id); new_context->setSetting("describe_extend_object_types", true); /// Expect only needed columns from the result of DESC TABLE. From 323128df6f3c779f3b2fe4a751fa98372a54fbbb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 15:02:32 +0200 Subject: [PATCH 181/226] Remove non existing ctor of Cluster::Address Signed-off-by: Azat Khuzhin --- src/Interpreters/Cluster.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index de10a445d010..b90acd1d576d 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -144,12 +144,6 @@ class Cluster UInt32 shard_index_ = 0, UInt32 replica_index_ = 0); - Address( - const String & host_port_, - const ClusterConnectionParameters & params, - UInt32 shard_index_, - UInt32 replica_index_); - Address( const DatabaseReplicaInfo & info, const ClusterConnectionParameters & params, From 4a33e027c518f51d120c60b21ccd962264e1356a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Jun 2023 17:31:45 +0200 Subject: [PATCH 182/226] Split StorageReplicatedMergeTree reading methods Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.cpp | 133 ++++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 32 ++++- 2 files changed, 115 insertions(+), 50 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 13c0fb3f7c25..4e053c4598ca 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4902,67 +4902,102 @@ void StorageReplicatedMergeTree::read( snapshot_data.alter_conversions = {}; }); - /** The `select_sequential_consistency` setting has two meanings: - * 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas. - * 2. Do not read parts that have not yet been written to the quorum of the replicas. - * For this you have to synchronously go to ZooKeeper. - */ - if (local_context->getSettingsRef().select_sequential_consistency) - { - 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*/false)) - query_plan = std::move(*plan); - return; - } + const auto & settings = local_context->getSettingsRef(); - if (local_context->canUseParallelReplicasOnInitiator()) - { - auto table_id = getStorageID(); + /// The `select_sequential_consistency` setting has two meanings: + /// 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas. + /// 2. Do not read parts that have not yet been written to the quorum of the replicas. + /// For this you have to synchronously go to ZooKeeper. + if (settings.select_sequential_consistency) + return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); - ASTPtr modified_query_ast; + if (local_context->canUseParallelReplicasOnInitiator()) + return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); - Block header; + readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); +} - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); +void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); + 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= */false); + if (plan) + query_plan = std::move(*plan); +} - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToSelectQuery(modified_query_tree); - } - else - { - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } +void StorageReplicatedMergeTree::readParallelReplicasImpl( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + const size_t /*max_block_size*/, + const size_t /*num_streams*/) +{ + auto table_id = getStorageID(); - auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + auto parallel_replicas_cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( - header, - {}, - storage_snapshot, - processed_stage); + ASTPtr modified_query_ast; + Block header; + if (local_context->getSettingsRef().allow_experimental_analyzer) + { + auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, - select_stream_factory, modified_query_ast, - local_context, query_info, cluster); + header = InterpreterSelectQueryAnalyzer::getSampleBlock( + modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_ast = queryNodeToSelectQuery(modified_query_tree); } 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); + modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + 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(), + /* table_func_ptr= */ nullptr, + select_stream_factory, modified_query_ast, + local_context, query_info, parallel_replicas_cluster); +} + +void StorageReplicatedMergeTree::readLocalImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const size_t num_streams) +{ + auto plan = reader.read( + column_names, storage_snapshot, query_info, + local_context, max_block_size, num_streams, + processed_stage, + /* max_block_numbers_to_read= */ nullptr, + /* enable_parallel_reading= */ local_context->canUseParallelReplicasOnFollower()); + if (plan) + query_plan = std::move(*plan); } template diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1a1b3c3b10c5..ded940bc1d2c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -130,7 +130,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; @@ -513,6 +513,36 @@ class StorageReplicatedMergeTree final : public MergeTreeData static std::optional distributedWriteFromClusterStorage(const std::shared_ptr & src_storage_cluster, const ASTInsertQuery & query, ContextPtr context); + void readLocalImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + + void readLocalSequentialConsistencyImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + + void readParallelReplicasImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + template void foreachActiveParts(Func && func, bool select_sequential_consistency) const; From ac54be9652414e10a1b79ec4f92439db5155310b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:56:18 +0200 Subject: [PATCH 183/226] Fix a test --- tests/integration/test_backward_compatibility/test_functions.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index fa24b146feca..c86c3ba0ab29 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -143,6 +143,7 @@ def test_string_functions(start_cluster): "position", "substring", "CAST", + "getTypeSerializationStreams", # NOTE: no need to ignore now()/now64() since they will fail because they don't accept any argument # 22.8 Backward Incompatible Change: Extended range of Date32 "toDate32OrZero", From 810137e57a53467e9fea668769749c559af12bc1 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 24 Jul 2023 05:59:07 +0000 Subject: [PATCH 184/226] Add new peak_memory_usage to docs --- docs/en/interfaces/http.md | 18 +++++++++--------- docs/ru/interfaces/http.md | 16 ++++++++-------- docs/zh/interfaces/http.md | 18 +++++++++--------- 3 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 3a7f6d4d8547..37821f0fee1d 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -56,7 +56,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -286,9 +286,9 @@ Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` Possible header fields: @@ -416,7 +416,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -581,7 +581,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -621,7 +621,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -673,7 +673,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -692,7 +692,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index b8c5ee77f0c4..981f1c7b5a21 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -50,7 +50,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -266,9 +266,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` Возможные поля заголовка: @@ -529,7 +529,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -569,7 +569,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -621,7 +621,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -640,7 +640,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index c7a0f355a92d..f84768beccc4 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -53,7 +53,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -262,9 +262,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 您可以在`X-ClickHouse-Progress`响应头中收到查询进度的信息。为此,启用[Http Header携带进度](../operations/settings/settings.md#settings-send_progress_in_http_headers)。示例: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` 显示字段信息: @@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -521,7 +521,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -561,7 +561,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -613,7 +613,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -632,7 +632,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact From ceaaa78fdcfac2243bcf28624336217bd44898f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:04:38 +0200 Subject: [PATCH 185/226] Fix transform --- src/Functions/transform.cpp | 37 ++++++++++++++++++------------------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 1fc0e3adf96c..a48d8d47489b 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -156,15 +156,15 @@ namespace { initialize(arguments, result_type); - const auto * in = arguments.front().column.get(); - - if (isColumnConst(*in)) + if (isColumnConst(*arguments[0].column)) return executeConst(arguments, result_type, input_rows_count); ColumnPtr default_non_const; if (!cache.default_column && arguments.size() == 4) default_non_const = castColumn(arguments[3], result_type); + ColumnPtr in = cache.default_column ? arguments[0].column : castColumn(arguments[0], result_type); + auto column_result = result_type->createColumn(); if (cache.is_empty) { @@ -174,30 +174,30 @@ namespace } else if (cache.table_num_to_idx) { - if (!executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const)) + if (!executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); } } else if (cache.table_string_to_idx) { - if (!executeString(in, *column_result, default_non_const)) - executeContiguous(in, *column_result, default_non_const); + if (!executeString(in.get(), *column_result, default_non_const)) + executeContiguous(in.get(), *column_result, default_non_const); } else if (cache.table_anything_to_idx) { - executeAnything(in, *column_result, default_non_const); + executeAnything(in.get(), *column_result, default_non_const); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); @@ -810,7 +810,6 @@ namespace cache.initialized = true; } }; - } REGISTER_FUNCTION(Transform) From aaa0bf64fd888332bfa59c284508d4e7a84d372c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:05:55 +0200 Subject: [PATCH 186/226] Add a test --- .../02832_transform_fixed_string_no_default.reference | 1 + .../0_stateless/02832_transform_fixed_string_no_default.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference create mode 100644 tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference new file mode 100644 index 000000000000..9daeafb9864c --- /dev/null +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference @@ -0,0 +1 @@ +test diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql new file mode 100644 index 000000000000..8d316d3413f2 --- /dev/null +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql @@ -0,0 +1 @@ +SELECT transform(name, ['a', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); From 890a3754a6a093545122e42bcab066a27c72ed5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:19:46 +0200 Subject: [PATCH 187/226] Fix error --- src/Functions/transform.cpp | 55 ++++++++++++++++++++----------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index a48d8d47489b..79168d82c54c 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -156,14 +156,18 @@ namespace { initialize(arguments, result_type); - if (isColumnConst(*arguments[0].column)) + const auto * in = arguments[0].column.get(); + + if (isColumnConst(*in)) return executeConst(arguments, result_type, input_rows_count); ColumnPtr default_non_const; if (!cache.default_column && arguments.size() == 4) default_non_const = castColumn(arguments[3], result_type); - ColumnPtr in = cache.default_column ? arguments[0].column : castColumn(arguments[0], result_type); + ColumnPtr in_casted = arguments[0].column; + if (arguments.size() == 3) + in_casted = castColumn(arguments[0], result_type); auto column_result = result_type->createColumn(); if (cache.is_empty) @@ -174,30 +178,30 @@ namespace } else if (cache.table_num_to_idx) { - if (!executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const)) + if (!executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); } } else if (cache.table_string_to_idx) { - if (!executeString(in.get(), *column_result, default_non_const)) - executeContiguous(in.get(), *column_result, default_non_const); + if (!executeString(in, *column_result, default_non_const, *in_casted)) + executeContiguous(in, *column_result, default_non_const, *in_casted); } else if (cache.table_anything_to_idx) { - executeAnything(in.get(), *column_result, default_non_const); + executeAnything(in, *column_result, default_non_const, *in_casted); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); @@ -218,7 +222,7 @@ namespace return impl->execute(args, result_type, input_rows_count); } - void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const size_t size = in->size(); const auto & table = *cache.table_anything_to_idx; @@ -236,11 +240,11 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } - void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const size_t size = in->size(); const auto & table = *cache.table_string_to_idx; @@ -255,12 +259,12 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } template - bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const + bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -297,7 +301,7 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } return true; @@ -451,7 +455,7 @@ namespace } } - bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const + bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -488,7 +492,7 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, 0); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } return true; @@ -810,6 +814,7 @@ namespace cache.initialized = true; } }; + } REGISTER_FUNCTION(Transform) From c79492240194f0d5dd9053c70a967c39a7536cb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:20:30 +0200 Subject: [PATCH 188/226] More tests --- .../02832_transform_fixed_string_no_default.reference | 2 ++ .../0_stateless/02832_transform_fixed_string_no_default.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference index 9daeafb9864c..ea545c90391a 100644 --- a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference @@ -1 +1,3 @@ test + +\N diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql index 8d316d3413f2..0e58c716c9fa 100644 --- a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql @@ -1 +1,3 @@ SELECT transform(name, ['a', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); +SELECT transform(name, ['test', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); +SELECT transform(name, ['a', 'test'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); From 0e46cf86b772e1513d837d6019181a6d291b7219 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 24 Jul 2023 08:52:19 +0200 Subject: [PATCH 189/226] Added try-except to check cases when second backup/restore is picked up first --- .../test_disallow_concurrency.py | 69 +++++++++++++++---- 1 file changed, 57 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index d0ce2e030169..a863a6e20475 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -133,9 +133,21 @@ def test_concurrent_backups_on_same_node(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + # It is possible that the second backup was picked up first, and then the async backup + if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + return + else: + raise e expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -179,9 +191,20 @@ def test_concurrent_backups_on_different_nodes(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + except Exception as e: + status = ( + nodes[1] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + return + else: + raise e expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -224,9 +247,20 @@ def test_concurrent_restores_on_same_node(): ) assert status in ["RESTORING", "RESTORED"] - error = nodes[0].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "RESTORING" or status == "RESTORE_FAILED": + return + else: + raise e expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", @@ -269,9 +303,20 @@ def test_concurrent_restores_on_different_node(): ) assert status in ["RESTORING", "RESTORED"] - error = nodes[1].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) + try: + error = nodes[1].query_and_get_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "RESTORING" or status == "RESTORE_FAILED": + return + else: + raise e expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", From 2471b032ab7a504d1997e9d3681bf97f0564273d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 09:52:49 +0000 Subject: [PATCH 190/226] fix lightweight delete after drop of projection --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- .../02792_drop_projection_lwd.reference | 1 + .../0_stateless/02792_drop_projection_lwd.sql | 26 +++++++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02792_drop_projection_lwd.reference create mode 100644 tests/queries/0_stateless/02792_drop_projection_lwd.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d773f3803774..06a9b62d9dec 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5693,7 +5693,8 @@ bool MergeTreeData::supportsLightweightDelete() const auto lock = lockParts(); for (const auto & part : data_parts_by_info) { - if (!part->supportLightweightDeleteMutate()) + if (part->getState() == MergeTreeDataPartState::Active + && !part->supportLightweightDeleteMutate()) return false; } return true; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.reference b/tests/queries/0_stateless/02792_drop_projection_lwd.reference new file mode 100644 index 000000000000..6529ff889b0c --- /dev/null +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.reference @@ -0,0 +1 @@ +98 diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql new file mode 100644 index 000000000000..fd446a8efe8b --- /dev/null +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS t_projections_lwd; + +CREATE TABLE t_projections_lwd (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); + +-- LWD works +DELETE FROM t_projections_lwd WHERE a = 0; + +-- add projection +ALTER TABLE t_projections_lwd ADD PROJECTION p_t_projections_lwd (SELECT * ORDER BY b); +ALTER TABLE t_projections_lwd MATERIALIZE PROJECTION p_t_projections_lwd; + +-- LWD does not work, as expected +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError UNFINISHED } +KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; + +-- drop projection +SET mutations_sync = 2; +ALTER TABLE t_projections_lwd DROP projection p_t_projections_lwd; + +DELETE FROM t_projections_lwd WHERE a = 2; + +SELECT count() FROM t_projections_lwd; + +DROP TABLE t_projections_lwd; From 79cc81890316338e35f13576cfd0360494e72645 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 11:06:21 +0000 Subject: [PATCH 191/226] try to fix test --- .../02726_async_insert_flush_stress.sh | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 5fafb773d164..876766d0780c 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -11,7 +11,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function insert1() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV 1,"a" 2,"b" @@ -22,7 +24,9 @@ function insert1() function insert2() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' done } @@ -30,28 +34,33 @@ function insert2() function insert3() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" done } function select1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" done } function select2() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" done } function flush1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do sleep 0.2 ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" done @@ -70,14 +79,14 @@ export -f select2 export -f flush1 for _ in {1..5}; do - timeout $TIMEOUT bash -c insert1 & - timeout $TIMEOUT bash -c insert2 & - timeout $TIMEOUT bash -c insert3 & + insert1 $TIMEOUT & + insert2 $TIMEOUT & + insert3 $TIMEOUT & done -timeout $TIMEOUT bash -c select1 & -timeout $TIMEOUT bash -c select2 & -timeout $TIMEOUT bash -c flush1 & +select1 $TIMEOUT & +select2 $TIMEOUT & +flush1 $TIMEOUT & wait From c7239c64ea36a6994cd88d34edc3774243472a68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 24 Jul 2023 15:16:44 +0200 Subject: [PATCH 192/226] Remove unused code --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 48adf36e6783..3eba9a9de24a 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -145,9 +145,6 @@ bool IMergeTreeSelectAlgorithm::getNewTask() ChunkAndProgress IMergeTreeSelectAlgorithm::read() { - size_t num_read_rows = 0; - size_t num_read_bytes = 0; - while (!is_cancelled) { try @@ -178,10 +175,6 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() ordered_columns.push_back(res.block.getByName(name).column); } - /// Account a progress from previous empty chunks. - res.num_read_rows += num_read_rows; - res.num_read_bytes += num_read_bytes; - return ChunkAndProgress{ .chunk = Chunk(ordered_columns, res.row_count), .num_read_rows = res.num_read_rows, @@ -194,7 +187,7 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() } } - return {Chunk(), num_read_rows, num_read_bytes, true}; + return {Chunk(), 0, 0, true}; } void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( From d2d100b68a4fc1765708a276b217faf403722fb4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jul 2023 17:05:57 +0200 Subject: [PATCH 193/226] Cancel execution in PipelineExecutor in case of exception in graph->updateNode --- src/Processors/Executors/PipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index f523e7b7cf9a..1508d834592f 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -272,7 +272,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie /// Prepare processor after execution. if (!graph->updateNode(context.getProcessorID(), queue, async_queue)) - finish(); + cancel(); /// Push other tasks to global queue. tasks.pushTasks(queue, async_queue, context); From f067f8c46d2aec217c3f835441ca1a2a281c72fd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 24 Jul 2023 15:37:16 +0000 Subject: [PATCH 194/226] Make 01951_distributed_push_down_limit analyzer agnostic --- tests/analyzer_tech_debt.txt | 1 - ...1951_distributed_push_down_limit.reference | 32 +++++++++---------- .../01951_distributed_push_down_limit.sql | 4 +-- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index b746d1610a49..1d56b2c3a717 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -72,7 +72,6 @@ 01925_test_storage_merge_aliases 01930_optimize_skip_unused_shards_rewrite_in 01947_mv_subquery -01951_distributed_push_down_limit 01952_optimize_distributed_group_by_sharding_key 02000_join_on_const 02001_shard_num_shard_count diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index b9a7d17e9554..d175d31846bc 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -1,19 +1,19 @@ -- { echo } -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Merge sorted streams after aggregation stage for ORDER BY) +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +Expression + Limit + Sorting Union - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Merge sorted streams after aggregation stage for ORDER BY) + Sorting + Expression + ReadFromStorage + ReadFromRemote +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +Expression + Limit + Sorting Union - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) + Sorting + Expression + ReadFromStorage + ReadFromRemote diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql index 184e63219881..aee714a494ed 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql @@ -3,5 +3,5 @@ set prefer_localhost_replica = 1; -- { echo } -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; From 0bbf26549f4fb49c599b4a58475c71bccfe9b37b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 18:13:15 +0200 Subject: [PATCH 195/226] Fix test --- tests/integration/test_drop_is_lock_free/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 8d92d7842261..9f595800beab 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -104,7 +104,7 @@ def test_query_is_lock_free(lock_free_query, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table}; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; """, query_id=query_id, ) @@ -173,7 +173,7 @@ def test_query_is_permanent(transaction, permanent, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table}; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; """, query_id=query_id, ) From 032956dd1eeca994d6fa5a66f974cfa10203c205 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 18:42:02 +0200 Subject: [PATCH 196/226] fix --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../test.py | 27 ++++++++++--------- .../01111_create_drop_replicated_db_stress.sh | 2 +- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 2393f45ebb65..e11913fc3d27 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -576,7 +576,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// It's ok if replica became readonly due to connection loss after we got current zookeeper (in this case zookeeper must be expired). /// And it's ok if replica became readonly after shutdown. /// In other cases it's likely that someone called pullLogsToQueue(...) when queue is not initialized yet by RestartingThread. - bool not_completely_initialized = storage.is_readonly && !zookeeper->expired() && !storage.shutdown_called; + bool not_completely_initialized = storage.is_readonly && !zookeeper->expired() && !storage.shutdown_prepared_called; if (not_completely_initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Tried to pull logs to queue (reason: {}) on readonly replica {}, it's a bug", reason, storage.getStorageID().getNameForLogs()); diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index a2a4ec92cf77..20b6a6c977f1 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -49,25 +49,28 @@ def soft_shutdown(node): node.stop_clickhouse(kill=False, stop_wait_sec=60) p = Pool(50) - pm = PartitionManager() - - pm.partition_instances(node1, node2) def insert(value): node1.query(f"INSERT INTO test_table VALUES ({value})") - p.map(insert, range(1, 50)) + with PartitionManager() as pm: + pm.partition_instances(node1, node2) + # iptables rules must be applied immediately, but looks like sometimes they are not... + time.sleep(3) - # Start shutdown async - waiter = p.apply_async(soft_shutdown, (node1,)) - # to be sure that shutdown started - time.sleep(5) + p.map(insert, range(1, 50)) - # node 2 partitioned and don't see any data - assert node2.query("SELECT * FROM test_table") == "0\n" + # Start shutdown async + waiter = p.apply_async(soft_shutdown, (node1,)) + # to be sure that shutdown started + time.sleep(5) + + # node 2 partitioned and don't see any data + assert node2.query("SELECT * FROM test_table") == "0\n" + + # Restore network + pm.heal_all() - # Restore network - pm.heal_all() # wait for shutdown to finish waiter.get() diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index cc63af3676b4..59899e1c14a7 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -56,7 +56,7 @@ function create_table() if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ "create table $database.rmt_${RANDOM}_${RANDOM}_${RANDOM} (n int) engine=ReplicatedMergeTree order by tuple() -- suppress $CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" \ - 2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE + 2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv TABLE_IS_DROPPED sleep 0.$RANDOM done } From 22a2fa097f3795cb2a483e899482b97f80aa8189 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 19:40:02 +0200 Subject: [PATCH 197/226] Improve error messages --- src/Functions/GregorianDate.cpp | 2 +- src/Functions/parseDateTime.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index aaaeeb7339d5..f28194781c22 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -125,7 +125,7 @@ void GregorianDate::init(ReadBuffer & in) assertEOF(in); if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year(year_), month_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date, out of range (year: {}, month: {}, day_of_month: {})."); } bool GregorianDate::tryInit(ReadBuffer & in) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index c3fbc08c4a95..2381def91515 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -398,7 +398,7 @@ namespace static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, out of range (year: {} day of year: {})", year_, day_of_year_); Int32 res = daysSinceEpochFromDate(year_, 1, 1); res += day_of_year_ - 1; @@ -408,7 +408,7 @@ namespace static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, out of range (year: {} month: {} day_of_month: {})", year_, month_, day_); Int32 res = cumulativeYearDays[year_ - 1970]; res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; From 654af41431423907fdffed93287e9160f78698b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 19:45:55 +0200 Subject: [PATCH 198/226] Fix race --- src/Functions/transform.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 79168d82c54c..e03701327b13 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -658,13 +658,13 @@ namespace std::unique_ptr table_string_to_idx; std::unique_ptr table_anything_to_idx; - bool is_empty = false; - ColumnPtr from_column; ColumnPtr to_column; ColumnPtr default_column; - std::atomic initialized{false}; + bool is_empty = false; + bool initialized = false; + std::mutex mutex; }; @@ -697,13 +697,12 @@ namespace /// Can be called from different threads. It works only on the first call. void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const { + std::lock_guard lock(cache.mutex); if (cache.initialized) return; const DataTypePtr & from_type = arguments[0].type; - std::lock_guard lock(cache.mutex); - if (from_type->onlyNull()) { cache.is_empty = true; From c35da36ff2b78dff5b964774673b8c713aa22e95 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jul 2023 19:50:53 +0200 Subject: [PATCH 199/226] Fix default value --- base/poco/Foundation/include/Poco/URI.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index f4505147cedd..eba8109253db 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ class Foundation_API URI URI(); /// Creates an empty URI. - explicit URI(const std::string & uri, bool disable_url_encoding = true); + explicit URI(const std::string & uri, bool disable_url_encoding = false); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -362,7 +362,7 @@ class Foundation_API URI std::string _query; std::string _fragment; - bool _disable_url_encoding = true; + bool _disable_url_encoding = false; }; From 2f99363db0356f146db427934b63e9158b7b9858 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 20:51:53 +0300 Subject: [PATCH 200/226] Update 02136_scalar_subquery_metrics.sql --- tests/queries/0_stateless/02136_scalar_subquery_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql index 180610288aaa..17ff367a58dc 100644 --- a/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql +++ b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql @@ -6,7 +6,7 @@ SELECT '#02136_scalar_subquery_4', (SELECT max(number) FROM numbers(1000)) as n SYSTEM FLUSH LOGS; SELECT read_rows, query FROM system.query_log WHERE - event_date > yesterday() + event_date >= yesterday() AND type = 'QueryFinish' AND current_database == currentDatabase() AND query LIKE 'SELECT ''#02136_scalar_subquery_%' From ab086f15d09048deb30bef84d5d3e7e62fefd898 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Jun 2023 21:09:25 +0200 Subject: [PATCH 201/226] try to push down more --- src/Processors/QueryPlan/JoinStep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 2ff8f161e992..33fa7955e0d9 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -54,7 +54,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines bool JoinStep::allowPushDownToRight() const { - return join->pipelineType() == JoinPipelineType::YShaped; + return join->pipelineType() == JoinPipelineType::YShaped || join->pipelineType() == JoinPipelineType::FillRightFirst; } void JoinStep::describePipeline(FormatSettings & settings) const From b2acbe42b722f83c0ffde1c8697e5f19bb14747f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Jun 2023 21:15:19 +0200 Subject: [PATCH 202/226] add perf test --- tests/performance/join_filter_pushdown.xml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/performance/join_filter_pushdown.xml diff --git a/tests/performance/join_filter_pushdown.xml b/tests/performance/join_filter_pushdown.xml new file mode 100644 index 000000000000..3adbbb3029e7 --- /dev/null +++ b/tests/performance/join_filter_pushdown.xml @@ -0,0 +1,9 @@ + + create table t(a UInt64) engine=MergeTree order by tuple() + insert into t select * from numbers_mt(5e6) + + select * from t as t0 inner join t as t1 using(a) where t1.a = 100 + + drop table t + + From d0894532feff599d1e73acca1a9010a53a26b004 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 26 Jun 2023 21:17:34 +0200 Subject: [PATCH 203/226] fix --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 4 ++++ .../queries/0_stateless/02514_analyzer_drop_join_on.reference | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 4336de41b7bb..af47b6ff4cdb 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -341,6 +341,10 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) return 0; + /// There is no ASOF Right join, so we're talking about pushing to the right side + if (kind == JoinKind::Right && table_join.strictness() == JoinStrictness::Asof) + return 0; + bool is_left = kind == JoinKind::Left; const auto & input_header = is_left ? child->getInputStreams().front().header : child->getInputStreams().back().header; const auto & res_header = child->getOutputStream().header; diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 0037ab85c07a..1b177b84afa8 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -107,7 +107,7 @@ Header: bx String bx_0 String c2_5 String c1_3 UInt64 - Filter (( + (JOIN actions + DROP unused columns after JOIN))) + Expression Header: a2_6 String bx_0 String c2_5 String @@ -139,7 +139,7 @@ Header: bx String ReadFromMemoryStorage Header: b1 UInt64 b2 String - Expression ((JOIN actions + Change column names to column identifiers)) + Filter (( + (JOIN actions + Change column names to column identifiers))) Header: c1_3 UInt64 c2_5 String ReadFromMemoryStorage From 104d3bbbae82309d7d55d3a46a28e6f791791fba Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 21:10:00 +0200 Subject: [PATCH 204/226] add test --- .../0_stateless/01763_filter_push_down_bugs.sql | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 9a5ef4727c53..8470b4a33792 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -66,3 +66,17 @@ EXPLAIN indexes=1 SELECT id, delete_time FROM t1 DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; + +-- expected to get row (1, 3, 1, 4) from JOIN and empty result from the query +SELECT * +FROM +( + SELECT * + FROM Values('id UInt64, t UInt64', (1, 3)) +) AS t1 +ASOF INNER JOIN +( + SELECT * + FROM Values('id UInt64, t UInt64', (1, 1), (1, 2), (1, 3), (1, 4), (1, 5)) +) AS t2 ON (t1.id = t2.id) AND (t1.t < t2.t) +WHERE t2.t != 4; From edc479bbf8c72a7076b092dd880fc7d8d2252e4d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 23:40:27 +0200 Subject: [PATCH 205/226] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +++- src/Storages/StorageReplicatedMergeTree.h | 3 +++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6cdcffab50ad..e64319278053 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,6 +4861,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } + std::lock_guard lock{flush_and_shutdown_mutex}; if (shutdown_prepared_called.load() || shutdown_called.load()) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); @@ -4906,6 +4907,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) void StorageReplicatedMergeTree::flushAndPrepareForShutdown() { + std::lock_guard lock{flush_and_shutdown_mutex}; if (shutdown_prepared_called.exchange(true)) return; @@ -4922,7 +4924,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); - /// Explicetly set the event, because the restarting thread will not set it again + /// Explicitly set the event, because the restarting thread will not set it again startup_event.set(); shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1c721e3724be..daa39536fa70 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -481,6 +481,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData std::atomic shutdown_prepared_called {false}; std::optional shutdown_deadline; + /// We call flushAndPrepareForShutdown before acquiring DDLGuard, so we can shutdown a table that is being created right now + mutable std::mutex flush_and_shutdown_mutex; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; From 21382afa2b2c686cde3ac0702b548d872373d3b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Jul 2023 06:10:04 +0200 Subject: [PATCH 206/226] Check for punctuation --- .../AggregateFunctionSimpleLinearRegression.cpp | 2 +- src/Common/ConcurrentBoundedQueue.h | 2 +- src/Common/parseRemoteDescription.cpp | 16 ++-------------- src/Common/parseRemoteDescription.h | 2 +- src/Common/tests/gtest_sensitive_data_masker.cpp | 6 +++--- src/Coordination/ZooKeeperDataReader.cpp | 2 +- src/Core/tests/gtest_settings.cpp | 3 +-- src/DataTypes/NumberTraits.h | 2 +- .../getDictionaryConfigurationFromAST.cpp | 2 +- src/Functions/FunctionsStringHash.cpp | 5 ++--- src/Functions/GatherUtils/sliceHasImplAnyAll.h | 4 ++-- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/Interpreters/Aggregator.h | 2 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterRenameQuery.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Parsers/Kusto/ParserKQLOperators.h | 16 ++++++++-------- src/Parsers/tests/gtest_Parser.cpp | 4 ++-- .../Formats/Impl/ArrowFieldIndexUtil.h | 2 +- .../Formats/Impl/JSONEachRowRowInputFormat.cpp | 4 ++-- .../QueryPlan/IntersectOrExceptStep.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 2 +- src/Server/HTTPHandler.cpp | 2 +- src/Storages/StorageFile.cpp | 4 ++-- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/TableFunctions/TableFunctionFactory.cpp | 2 +- utils/check-style/check-style | 3 +++ 28 files changed, 45 insertions(+), 56 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp index 1ed6c83af7da..1489db558579 100644 --- a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp +++ b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp @@ -66,7 +66,7 @@ AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression( #undef FOR_LEASTSQR_TYPES #undef DISPATCH - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT , + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types ({}, {}) of arguments of aggregate function {}, must " "be Native Ints, Native UInts or Floats", x_arg->getName(), y_arg->getName(), name); } diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 01910c4caff1..922607da813f 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -110,7 +110,7 @@ class ConcurrentBoundedQueue /// Returns false if queue is finished [[nodiscard]] bool pushFront(const T & x) { - return emplaceImpl(/* timeout_milliseconds= */ std::nullopt , x); + return emplaceImpl(/* timeout_milliseconds= */ std::nullopt, x); } /// Returns false if queue is finished diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 0bcd62d30c7e..8ea3f4a0aa5f 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -52,20 +52,8 @@ static bool parseNumber(const String & description, size_t l, size_t r, size_t & } -/* Parse a string that generates shards and replicas. Separator - one of two characters | or , - * depending on whether shards or replicas are generated. - * For example: - * host1,host2,... - generates set of shards from host1, host2, ... - * host1|host2|... - generates set of replicas from host1, host2, ... - * abc{8..10}def - generates set of shards abc8def, abc9def, abc10def. - * abc{08..10}def - generates set of shards abc08def, abc09def, abc10def. - * abc{x,yy,z}def - generates set of shards abcxdef, abcyydef, abczdef. - * abc{x|yy|z} def - generates set of replicas abcxdef, abcyydef, abczdef. - * abc{1..9}de{f,g,h} - is a direct product, 27 shards. - * abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas. - */ -std::vector -parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name) +std::vector parseRemoteDescription( + const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name) { std::vector res; std::vector cur; diff --git a/src/Common/parseRemoteDescription.h b/src/Common/parseRemoteDescription.h index e3e4a3f523c2..d97558c47283 100644 --- a/src/Common/parseRemoteDescription.h +++ b/src/Common/parseRemoteDescription.h @@ -3,7 +3,7 @@ #include namespace DB { -/* Parse a string that generates shards and replicas. Separator - one of two characters | or , +/* Parse a string that generates shards and replicas. Separator - one of two characters '|' or ',' * depending on whether shards or replicas are generated. * For example: * host1,host2,... - generates set of shards from host1, host2, ... diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index 92c4edbac2a5..f36c41546845 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -27,7 +27,7 @@ TEST(Common, SensitiveDataMasker) { Poco::AutoPtr empty_xml_config = new Poco::Util::XMLConfiguration(); - DB::SensitiveDataMasker masker(*empty_xml_config , ""); + DB::SensitiveDataMasker masker(*empty_xml_config, ""); masker.addMaskingRule("all a letters", "a+", "--a--"); masker.addMaskingRule("all b letters", "b+", "--b--"); masker.addMaskingRule("all d letters", "d+", "--d--"); @@ -45,7 +45,7 @@ TEST(Common, SensitiveDataMasker) masker.printStats(); #endif - DB::SensitiveDataMasker masker2(*empty_xml_config , ""); + DB::SensitiveDataMasker masker2(*empty_xml_config, ""); masker2.addMaskingRule("hide root password", "qwerty123", "******"); masker2.addMaskingRule("hide SSN", "[0-9]{3}-[0-9]{2}-[0-9]{4}", "000-00-0000"); masker2.addMaskingRule("hide email", "[A-Za-z0-9._%+-]+@[A-Za-z0-9.-]+\\.[A-Za-z]{2,4}", "hidden@hidden.test"); @@ -58,7 +58,7 @@ TEST(Common, SensitiveDataMasker) "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', '******') WHERE " "ssn='000-00-0000' or email='hidden@hidden.test'"); - DB::SensitiveDataMasker maskerbad(*empty_xml_config , ""); + DB::SensitiveDataMasker maskerbad(*empty_xml_config, ""); // gtest has not good way to check exception content, so just do it manually (see https://github.com/google/googletest/issues/952 ) try diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 94fc07bcc4a3..79929c4e66e3 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -40,7 +40,7 @@ void deserializeSnapshotMagic(ReadBuffer & in) Coordination::read(dbid, in); static constexpr int32_t SNP_HEADER = 1514885966; /// "ZKSN" if (magic_header != SNP_HEADER) - throw Exception(ErrorCodes::CORRUPTED_DATA ,"Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header); + throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header); } int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in) diff --git a/src/Core/tests/gtest_settings.cpp b/src/Core/tests/gtest_settings.cpp index cbeb84ef2e7f..a6d8763bfb80 100644 --- a/src/Core/tests/gtest_settings.cpp +++ b/src/Core/tests/gtest_settings.cpp @@ -121,7 +121,7 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetString) ASSERT_EQ(Field("decimal,datetime64"), setting); // comma with spaces - setting = " datetime64 , decimal "; + setting = " datetime64 , decimal "; /// bad punctuation is ok here ASSERT_TRUE(setting.changed); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DECIMAL)); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DATETIME64)); @@ -166,4 +166,3 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetInvalidString) ASSERT_TRUE(setting.changed); ASSERT_EQ(0, setting.value.getValue()); } - diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index 6b068b0d8b1f..cf283d3358c3 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -174,7 +174,7 @@ template struct ResultOfBitNot * Float, [U]Int -> Float * Decimal, Decimal -> Decimal * UUID, UUID -> UUID - * UInt64 , Int -> Error + * UInt64, Int -> Error * Float, [U]Int64 -> Error */ template diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0b7352e9cbb6..b12ffc555d4d 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -322,7 +322,7 @@ void buildSingleAttribute( /** Transforms - * PRIMARY KEY Attr1 ,..., AttrN + * PRIMARY KEY Attr1, ..., AttrN * to the next configuration * Attr1 * or diff --git a/src/Functions/FunctionsStringHash.cpp b/src/Functions/FunctionsStringHash.cpp index d6873d9490e0..ff8ff2d26517 100644 --- a/src/Functions/FunctionsStringHash.cpp +++ b/src/Functions/FunctionsStringHash.cpp @@ -292,8 +292,8 @@ struct SimHashImpl continue; // we need to store the new word hash value to the oldest location. - // for example, N = 5, array |a0|a1|a2|a3|a4|, now , a0 is the oldest location, - // so we need to store new word hash into location of a0, then ,this array become + // for example, N = 5, array |a0|a1|a2|a3|a4|, now, a0 is the oldest location, + // so we need to store new word hash into location of a0, then this array become // |a5|a1|a2|a3|a4|, next time, a1 become the oldest location, we need to store new // word hash value into location of a1, then array become |a5|a6|a2|a3|a4| words[offset] = BytesRef{word_start, length}; @@ -793,4 +793,3 @@ REGISTER_FUNCTION(StringHash) factory.registerFunction(); } } - diff --git a/src/Functions/GatherUtils/sliceHasImplAnyAll.h b/src/Functions/GatherUtils/sliceHasImplAnyAll.h index 21c80b742fdb..99bf1a7cc334 100644 --- a/src/Functions/GatherUtils/sliceHasImplAnyAll.h +++ b/src/Functions/GatherUtils/sliceHasImplAnyAll.h @@ -375,14 +375,14 @@ bool sliceHasImplAnyAllImplInt16( _mm256_or_si256( _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)), - _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)))), + _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)))), _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6)), _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6))))), _mm256_or_si256( _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)), - _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data ,1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)))), + _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)))), _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2)), _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2)))))) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1a367a8199d6..fd825720ac95 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -258,7 +258,7 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, - Aws::Utils::RateLimits::RateLimiterInterface * readLimiter , + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 05b34e8460fc..29096a38be62 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -292,7 +292,7 @@ struct AggregationMethodStringNoCache { } - using State = ColumnsHashing::HashMethodString; + using State = ColumnsHashing::HashMethodString; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = nullable; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 193bb5b6ab0f..92e6bcb326cd 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -551,7 +551,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) chassert(!task.completely_processed); /// Setup tracing context on current thread for current DDL - OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__ , + OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__, task.entry.tracing_context, this->context->getOpenTelemetrySpanLog()); tracing_ctx_holder.root_span.kind = OpenTelemetry::CONSUMER; diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 75d43b541e15..ae79b3f932ec 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -193,7 +193,7 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.getDatabase(), elem.to.getTable()); if (rename.exchange) { - required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT , elem.from.getDatabase(), elem.from.getTable()); + required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.getDatabase(), elem.from.getTable()); required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.getDatabase(), elem.to.getTable()); } } diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 6257e617d4ac..2ef4f4d62183 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -482,7 +482,7 @@ CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN bool removed = running_list.erase(txn->tid.getHash()); if (!removed) { - LOG_ERROR(log , "I's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid); + LOG_ERROR(log, "It's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid); abort(); } } diff --git a/src/Parsers/Kusto/ParserKQLOperators.h b/src/Parsers/Kusto/ParserKQLOperators.h index 9796ae10c07c..72e25cc3cf90 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.h +++ b/src/Parsers/Kusto/ParserKQLOperators.h @@ -31,10 +31,10 @@ class KQLOperators not_endswith, endswith_cs, not_endswith_cs, - equal, //=~ - not_equal,//!~ - equal_cs, //= - not_equal_cs,//!= + equal, /// =~ + not_equal, /// !~ + equal_cs, /// = + not_equal_cs, /// != has, not_has, has_all, @@ -49,10 +49,10 @@ class KQLOperators not_hassuffix, hassuffix_cs, not_hassuffix_cs, - in_cs, //in - not_in_cs, //!in - in, //in~ - not_in ,//!in~ + in_cs, /// in + not_in_cs, /// !in + in, /// in~ + not_in, /// !in~ matches_regex, startswith, not_startswith, diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index d77ae8d3a275..18e91c533e02 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -359,11 +359,11 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "SELECT *\nFROM Customers\nORDER BY LastName DESC" }, { - "Customers | order by Age desc , FirstName asc ", + "Customers | order by Age desc, FirstName asc ", "SELECT *\nFROM Customers\nORDER BY\n Age DESC,\n FirstName ASC" }, { - "Customers | order by Age asc , FirstName desc", + "Customers | order by Age asc, FirstName desc", "SELECT *\nFROM Customers\nORDER BY\n Age ASC,\n FirstName DESC" }, { diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index b7adaa35335f..676ce50d04ff 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -35,7 +35,7 @@ class ArrowFieldIndexUtil /// - key: field name with full path. eg. a struct field's name is like a.x.i /// - value: a pair, first value refers to this field's start index, second value refers to how many /// indices this field take. eg. - /// For a parquet schema {x: int , y: {i: int, j: int}}, the return will be + /// For a parquet schema {x: int, y: {i: int, j: int}}, the return will be /// - x: (0, 1) /// - y: (1, 2) /// - y.i: (1, 1) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index e5f52936021c..b1b08cdf256e 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -236,10 +236,10 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi bool JSONEachRowRowInputFormat::checkEndOfData(bool is_first_row) { - /// We consume , or \n before scanning a new row, instead scanning to next row at the end. + /// We consume ',' or '\n' before scanning a new row, instead scanning to next row at the end. /// The reason is that if we want an exact number of rows read with LIMIT x /// from a streaming table engine with text data format, like File or Kafka - /// then seeking to next ;, or \n would trigger reading of an extra row at the end. + /// then seeking to next ';,' or '\n' would trigger reading of an extra row at the end. /// Semicolon is added for convenience as it could be used at end of INSERT query. if (!in->eof()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index afdff44020f5..b132d27670d2 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -30,7 +30,7 @@ static Block checkHeaders(const DataStreams & input_streams_) } IntersectOrExceptStep::IntersectOrExceptStep( - DataStreams input_streams_ , Operator operator_ , size_t max_threads_) + DataStreams input_streams_, Operator operator_, size_t max_threads_) : header(checkHeaders(input_streams_)) , current_operator(operator_) , max_threads(max_threads_) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 7f7f9058f1bb..1b20778877df 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -72,7 +72,7 @@ struct ViewsData std::atomic_bool has_exception = false; std::exception_ptr first_exception; - ViewsData(ThreadStatusesHolderPtr thread_status_holder_, ContextPtr context_, StorageID source_storage_id_, StorageMetadataPtr source_metadata_snapshot_ , StoragePtr source_storage_) + ViewsData(ThreadStatusesHolderPtr thread_status_holder_, ContextPtr context_, StorageID source_storage_id_, StorageMetadataPtr source_metadata_snapshot_, StoragePtr source_storage_) : thread_status_holder(std::move(thread_status_holder_)) , context(std::move(context_)) , source_storage_id(std::move(source_storage_id_)) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 069670c84a56..29b75fa6552b 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -638,7 +638,7 @@ void HTTPHandler::processQuery( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected MemoryWriteBuffer"); auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); + copyData(*rdbuf, *next_buffer); return next_buffer; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cbd32460f7e0..3126d5849641 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -242,8 +242,8 @@ void listFilesWithRegexpMatchingImpl( { if (recursive) { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob , + listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "", + looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob, total_bytes_to_read, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 14b7fc15af27..582dc6f882d6 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,7 +149,7 @@ class StorageProxy : public IStorage return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - CheckResults checkData(const ASTPtr & query , ContextPtr context) override { return getNested()->checkData(query, context); } + CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4e053c4598ca..c3dedd69d0d6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6569,7 +6569,7 @@ void StorageReplicatedMergeTree::fetchPartition( try { - /// part name , metadata, part_path , true, 0, zookeeper + /// part name, metadata, part_path, true, 0, zookeeper if (!fetchPart(part_name, metadata_snapshot, from_zookeeper_name, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) throw Exception(ErrorCodes::UNFINISHED, "Failed to fetch part {} from {}", part_name, from_); } diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 76108f1cdd4c..ce3daff07859 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -41,7 +41,7 @@ TableFunctionPtr TableFunctionFactory::get( { auto hints = getHints(table_function->name); if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name , toString(hints)); + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name, toString(hints)); else throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function->name); } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 0b3b86b4772e..c28ca1cfc8a9 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -410,3 +410,6 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep # The stateful directory should only contain the tests that depend on the test dataset (hits or visits). find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep -v '00076_system_columns_bytes' | xargs -I{} bash -c 'grep -q -P "hits|visits" "{}" || echo "The test {} does not depend on the test dataset (hits or visits table) and should be located in the 0_stateless directory. You can also add an exception to the check-style script."' + +# Check for bad punctuation: whitespace before comma. +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'" From b02e290d5507419e6166433b0a045eaeb3d124d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:37:02 +0200 Subject: [PATCH 207/226] tests: fix 01035_avg_weighted_long flakiness Use one clickhouse-client invocation instead of 300, in debug builds it is significant - each spawn is ~1 second Signed-off-by: Azat Khuzhin --- .../0_stateless/01035_avg_weighted_long.sh | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.sh b/tests/queries/0_stateless/01035_avg_weighted_long.sh index 138aa03fbb35..8838b07a3d77 100755 --- a/tests/queries/0_stateless/01035_avg_weighted_long.sh +++ b/tests/queries/0_stateless/01035_avg_weighted_long.sh @@ -11,36 +11,36 @@ ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal256 ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal32(1, 0) x, toDecimal256(1, 1) y);" types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64") +exttypes=("Int128" "Int256" "UInt256") +# Decimal types +dtypes=("32" "64" "128" "256") -for left in "${types[@]}" -do - for right in "${types[@]}" +( + for left in "${types[@]}" do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" + for right in "${types[@]}" + do + echo "SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2));" + echo "SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0));" + done done -done - -exttypes=("Int128" "Int256" "UInt256") -for left in "${exttypes[@]}" -do - for right in "${exttypes[@]}" + for left in "${exttypes[@]}" do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(to${left}(1), to${right}(2))" + for right in "${exttypes[@]}" + do + echo "SELECT avgWeighted(to${left}(1), to${right}(2));" + done done -done - -# Decimal types -dtypes=("32" "64" "128" "256") -for left in "${dtypes[@]}" -do - for right in "${dtypes[@]}" + for left in "${dtypes[@]}" do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" + for right in "${dtypes[@]}" + do + echo "SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4));" + done done -done +) | clickhouse-client -nm echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* are non-conforming as arguments for aggregate function avgWeighted' From 2efbeab5afe50fbd734a6729e4cffa7ef12fff04 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:43:29 +0200 Subject: [PATCH 208/226] tests: fix 00719_parallel_ddl_table flakiness in debug builds In debug bulds each client invocation takes ~1 second, and on CI it can take more if the node is under some load, so let's decrease number of iterations. Anyway CI runs each test ~1K times daily, and if there will be something even this number of iterations should be enough. Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/00719_parallel_ddl_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00719_parallel_ddl_table.sh b/tests/queries/0_stateless/00719_parallel_ddl_table.sh index fdc994aec337..57a7e2283413 100755 --- a/tests/queries/0_stateless/00719_parallel_ddl_table.sh +++ b/tests/queries/0_stateless/00719_parallel_ddl_table.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl" function query() { - for _ in {1..100}; do + for _ in {1..50}; do ${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS parallel_ddl(a Int) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl" done From f8c90d5964a4c27dc119fd4417c23785a40b9c5e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 25 Jul 2023 13:36:57 +0200 Subject: [PATCH 209/226] Make better --- docs/en/sql-reference/transactions.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index 1ca2db44b130..cb89a091d684 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -5,7 +5,7 @@ slug: /en/guides/developer/transactional ## Case 1: INSERT into one partition, of one table, of the MergeTree* family -This is transactional (ACID) if the number of rows inserted is less than or equal to `max_insert_block_size rows`, and in the case of data in TSV, TKSV, CSV, or JSONEachRow format if the number of bytes is less than `min_chunk_bytes_for_parallel_parsing`: +This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen @@ -33,14 +33,16 @@ Same as Case 1 above, with this detail: - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes -- `max_insert_block_size` is 1 000 000 by default and can be adjusted as needed -- `min_chunk_bytes_for_parallel_parsing` is 1 000 000 by default and can be adjusted as needed +- rows inserted from the client in some data format are packed into a single block when: + - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) + - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data +- the size of the inserted block in general may depend on many settings (for example: `max_block_size`, `max_insert_block_size`, `min_insert_block_size_rows`, `min_insert_block_size_bytes`, `preferred_block_size_bytes`, etc) - if the client did not receive an answer from the server, the client does not know if the transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties - ClickHouse is using MVCC with snapshot isolation internally - all ACID properties are valid even in the case of server kill/crash - either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in the typical setup - "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) -- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback). +- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback) ## Transactions, Commit, and Rollback From 93e5d7f51c561af4d9236ef7e146b94754bc8fd8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Jul 2023 11:42:22 +0000 Subject: [PATCH 210/226] Fix flaky 00995_exception_while_insert --- tests/queries/0_stateless/00995_exception_while_insert.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00995_exception_while_insert.sh b/tests/queries/0_stateless/00995_exception_while_insert.sh index 927ac6a54e56..732dba6c6f1e 100755 --- a/tests/queries/0_stateless/00995_exception_while_insert.sh +++ b/tests/queries/0_stateless/00995_exception_while_insert.sh @@ -7,8 +7,8 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x = 1500000)) ENGINE = Memory;" -seq 1 2000000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non-zero." && echo 'OK' || echo 'FAIL' ||: +seq 1 1500000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non-zero." && echo 'OK' || echo 'FAIL' ||: $CLICKHOUSE_CLIENT --query="DROP TABLE check;" From 328d0a5269407eef6899907d6b9869307a56dfa4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 25 Jul 2023 14:50:27 +0200 Subject: [PATCH 211/226] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- .../test.py | 4 +++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e64319278053..9e4a63f6ba91 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,9 +4861,13 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } - std::lock_guard lock{flush_and_shutdown_mutex}; - if (shutdown_prepared_called.load() || shutdown_called.load()) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + auto lock = std::unique_lock(flush_and_shutdown_mutex, std::defer_lock); + do + { + if (shutdown_prepared_called.load() || shutdown_called.load()) + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + } + while (!lock.try_lock()); /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 20b6a6c977f1..d971e4ec6584 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -3,6 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry from multiprocessing.dummy import Pool import time @@ -54,9 +55,10 @@ def insert(value): node1.query(f"INSERT INTO test_table VALUES ({value})") with PartitionManager() as pm: + assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) # iptables rules must be applied immediately, but looks like sometimes they are not... - time.sleep(3) + assert_eq_with_retry(node1, "select count() from remote('node1,node2', 'system.one')", "1\n", settings={"skip_unavailable_shards": 1}) p.map(insert, range(1, 50)) From d7de8bf797a7444927e80c7c88d9b7c5a4040e01 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 25 Jul 2023 13:03:12 +0000 Subject: [PATCH 212/226] Automatic style fix --- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index d971e4ec6584..d1373d44d0fa 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -58,7 +58,12 @@ def insert(value): assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) # iptables rules must be applied immediately, but looks like sometimes they are not... - assert_eq_with_retry(node1, "select count() from remote('node1,node2', 'system.one')", "1\n", settings={"skip_unavailable_shards": 1}) + assert_eq_with_retry( + node1, + "select count() from remote('node1,node2', 'system.one')", + "1\n", + settings={"skip_unavailable_shards": 1}, + ) p.map(insert, range(1, 50)) From b91852de3a311cd03ef571e4470deba3deeba25b Mon Sep 17 00:00:00 2001 From: Julian Maicher Date: Tue, 25 Jul 2023 16:01:19 +0200 Subject: [PATCH 213/226] fix(docs): Document correct MODIFY COLUMN REMOVE syntax --- docs/en/sql-reference/statements/alter/column.md | 2 +- docs/ru/sql-reference/statements/alter/column.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index dae2c7dd1d3b..6ceb9b5849e7 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -213,7 +213,7 @@ Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC Syntax: ```sql -ALTER TABLE table_name MODIFY column_name REMOVE property; +ALTER TABLE table_name MODIFY COLUMN column_name REMOVE property; ``` **Example** diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index a8ace2130750..92be30b101a7 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -182,7 +182,7 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String) Синтаксис: ```sql -ALTER TABLE table_name MODIFY column_name REMOVE property; +ALTER TABLE table_name MODIFY COLUMN column_name REMOVE property; ``` **Пример** From bd09ad6736bac2b9e986993e75f1f8f61b1508a6 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Tue, 25 Jul 2023 16:19:44 +0200 Subject: [PATCH 214/226] MaterializedMySQL: Fix typos in tests --- .../materialized_with_ddl.py | 27 +++++++++++++------ .../test_materialized_mysql_database/test.py | 9 ++++--- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8b2943c2b733..389d430622d2 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -13,25 +13,36 @@ from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=10, interval_seconds=3): - lastest_result = "" +def check_query( + clickhouse_node, + query, + result_set, + retry_count=30, + interval_seconds=1, + on_failure=None, +): + latest_result = "" + if "/* expect: " not in query: + query = "/* expect: " + result_set.rstrip("\n") + "*/ " + query for i in range(retry_count): try: - lastest_result = clickhouse_node.query(query) - if result_set == lastest_result: + latest_result = clickhouse_node.query(query) + if result_set == latest_result: return - logging.debug(f"latest_result {lastest_result}") + logging.debug(f"latest_result {latest_result}") time.sleep(interval_seconds) except Exception as e: logging.debug(f"check_query retry {i+1} exception {e}") time.sleep(interval_seconds) else: - result_got = clickhouse_node.query(query) + latest_result = clickhouse_node.query(query) + if on_failure is not None and latest_result != result_set: + on_failure(latest_result, result_set) assert ( - result_got == result_set - ), f"Got result {result_got}, while expected result {result_set}" + latest_result == result_set + ), f"Got result '{latest_result}', expected result '{result_set}'" def dml_with_materialized_mysql_database(clickhouse_node, mysql_node, service_name): diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index c21e04af8dbf..1fd09f733f0b 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -52,6 +52,7 @@ def started_cluster(): cluster.start() yield cluster finally: + node_db.stop_clickhouse() # ensures that coverage report is written to disk, even if cluster.shutdown() times out. cluster.shutdown() @@ -86,7 +87,7 @@ def alloc_connection(self): else: self.mysql_connection.ping(reconnect=True) logging.debug( - "MySQL Connection establised: {}:{}".format( + "MySQL Connection established: {}:{}".format( self.ip_address, self.port ) ) @@ -94,7 +95,7 @@ def alloc_connection(self): except Exception as e: errors += [str(e)] time.sleep(1) - raise Exception("Connection not establised, {}".format(errors)) + raise Exception("Connection not established, {}".format(errors)) def query(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -118,9 +119,9 @@ def result(self, execution_query): if result is not None: print(cursor.fetchall()) - def query_and_get_data(self, executio_query): + def query_and_get_data(self, execution_query): with self.alloc_connection().cursor() as cursor: - cursor.execute(executio_query) + cursor.execute(execution_query) return cursor.fetchall() def close(self): From 2c7c38950d54c009e5268d371dabe8035b817283 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Jul 2023 14:21:12 +0000 Subject: [PATCH 215/226] better check for lightweight deletes --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- .../02792_drop_projection_lwd.reference | 2 +- .../0_stateless/02792_drop_projection_lwd.sql | 16 +++++----------- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 06a9b62d9dec..6179c70ca575 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5693,8 +5693,11 @@ bool MergeTreeData::supportsLightweightDelete() const auto lock = lockParts(); for (const auto & part : data_parts_by_info) { - if (part->getState() == MergeTreeDataPartState::Active - && !part->supportLightweightDeleteMutate()) + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (!part->supportLightweightDeleteMutate()) return false; } return true; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.reference b/tests/queries/0_stateless/02792_drop_projection_lwd.reference index 6529ff889b0c..3ad5abd03aea 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.reference +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.reference @@ -1 +1 @@ -98 +99 diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index fd446a8efe8b..a1d8a9c90f37 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -1,23 +1,17 @@ +SET mutations_sync = 2; + DROP TABLE IF EXISTS t_projections_lwd; -CREATE TABLE t_projections_lwd (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a; +CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER BY b)) ENGINE = MergeTree ORDER BY a; INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); --- LWD works -DELETE FROM t_projections_lwd WHERE a = 0; - --- add projection -ALTER TABLE t_projections_lwd ADD PROJECTION p_t_projections_lwd (SELECT * ORDER BY b); -ALTER TABLE t_projections_lwd MATERIALIZE PROJECTION p_t_projections_lwd; - -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError UNFINISHED } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError BAD_ARGUMENTS } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection -SET mutations_sync = 2; -ALTER TABLE t_projections_lwd DROP projection p_t_projections_lwd; +ALTER TABLE t_projections_lwd DROP projection p; DELETE FROM t_projections_lwd WHERE a = 2; From 79d0343becaa001dca587ee1932a8520e086d0ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 16:34:40 +0200 Subject: [PATCH 216/226] tests: fix 01821_join_table_race_long flakiness (#52559) By grouping multiple queries into one clickhouse-client invocation, since each execve of the binary can take ~1 second in debug builds. But this slightly changes the logic, so be aware. Signed-off-by: Azat Khuzhin Co-authored-by: Alexander Tokmakov --- tests/queries/0_stateless/01821_join_table_race_long.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh index e02fe7886530..561b856841bd 100755 --- a/tests/queries/0_stateless/01821_join_table_race_long.sh +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -9,13 +9,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" $CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM');"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & +for _ in {0..200}; do echo "SELECT count() FROM join_table_race FORMAT Null;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "TRUNCATE TABLE join_table_race;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & wait From 85082ad8f8ee0d1023273d8db888e143e59bd828 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 16:35:01 +0200 Subject: [PATCH 217/226] Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() (#52490) CI founds [1]: Exception: Sanitizer assert found for instance ================== WARNING: ThreadSanitizer: data race (pid=348) Write of size 8 at 0x7b58000044a0 by main thread: 2 DB::DatabaseReplicated::startupTables(ThreadPoolImpl>&, DB::LoadingStrictnessLevel) build_docker/./src/Databases/DatabaseReplicated.cpp:526:16 (clickhouse+0x1ec45092) 3 DB::TablesLoader::startupTables() build_docker/./src/Databases/TablesLoader.cpp:87:26 (clickhouse+0x1f9258ab) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 4 DB::loadMetadata(std::__1::shared_ptr, std::__1::basic_string, std::__1::allocator> const&) build_docker/./src/Interpreters/loadMetadata.cpp:234:12 (clickhouse+0x1fff3834) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 5 DB::Server::main() build_docker/./programs/server/Server.cpp:1615:9 (clickhouse+0x163e7f78) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 6 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x257608fe) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 7 DB::Server::run() build_docker/./programs/server/Server.cpp:391:25 (clickhouse+0x163d7d7c) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 8 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x25780114) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 9 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:196:20 (clickhouse+0x163d4c23) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 10 main build_docker/./programs/main.cpp:487:12 (clickhouse+0xdf8c877) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) Previous read of size 8 at 0x7b58000044a0 by thread T27 (mutexes: write M0, write M1): 1 DB::DatabaseReplicated::canExecuteReplicatedMetadataAlter() const build_docker/./src/Databases/DatabaseReplicated.cpp:1303:12 (clickhouse+0x1ec5c5bd) 2 DB::ReplicatedMergeTreeQueue::shouldExecuteLogEntry() const build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1471:24 (clickhouse+0x2115fb56) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 3 DB::ReplicatedMergeTreeQueue::selectEntryToProcess(DB::MergeTreeDataMergerMutator&, DB::MergeTreeData&) build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1676:13 (clickhouse+0x21163c58) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 4 DB::StorageReplicatedMergeTree::selectQueueEntry() build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3240:26 (clickhouse+0x20823db2) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 5 DB::StorageReplicatedMergeTree::scheduleDataProcessingJob(DB::BackgroundJobsAssignee&) build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3304:65 (clickhouse+0x208240fc) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) [1]: https://s3.amazonaws.com/clickhouse-test-reports/52395/0b258dda4ee618a4d002e2b5246d68bbd2c77c7e/integration_tests__tsan__[5_6].html Add ddl_worker_initialized flag to avoid this race. Note, that it should be enough to check this flag only in canExecuteReplicatedMetadataAlter() since only it can be run in parallel with ctor before it had been finished. v0: initialize ddl before startupTables() v2: ddl_worker_initialized Signed-off-by: Azat Khuzhin Co-authored-by: Alexander Tokmakov --- src/Databases/DatabaseReplicated.cpp | 4 +++- src/Databases/DatabaseReplicated.h | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d3b3d4b545fd..ed56edd7503a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -524,6 +524,7 @@ void DatabaseReplicated::startupTables(ThreadPool & thread_pool, LoadingStrictne ddl_worker = std::make_unique(this, getContext()); ddl_worker->startup(); + ddl_worker_initialized = true; } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const @@ -1155,6 +1156,7 @@ void DatabaseReplicated::stopReplication() void DatabaseReplicated::shutdown() { stopReplication(); + ddl_worker_initialized = false; ddl_worker = nullptr; DatabaseAtomic::shutdown(); } @@ -1299,7 +1301,7 @@ bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const /// It may update the metadata digest (both locally and in ZooKeeper) /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. /// We should not update metadata until the database is initialized. - return ddl_worker && ddl_worker->isCurrentlyActive(); + return ddl_worker_initialized && ddl_worker->isCurrentlyActive(); } void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8e33f482ac12..7ba91e480851 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -134,6 +134,7 @@ class DatabaseReplicated : public DatabaseAtomic std::atomic_bool is_readonly = true; std::atomic_bool is_probably_dropped = false; std::atomic_bool is_recovering = false; + std::atomic_bool ddl_worker_initialized = false; std::unique_ptr ddl_worker; UInt32 max_log_ptr_at_creation = 0; From c75b5bc740cd20ee7f5e6bb5a71b9f8e215eb03c Mon Sep 17 00:00:00 2001 From: Sanjam Panda <36253777+saitama951@users.noreply.github.com> Date: Tue, 25 Jul 2023 20:12:22 +0530 Subject: [PATCH 218/226] Update TwoLevelStringHashTable.h --- .../HashTable/TwoLevelStringHashTable.h | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index ee6dcd05d9ae..0527ec67e6ed 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -114,18 +114,18 @@ class TwoLevelStringHashTable : private boost::noncopyable { memcpy(&n[0], p, 8); if constexpr (std::endian::native == std::endian::little) - n[0] &= -1ULL >> s; - else - n[0] &= -1ULL << s; + n[0] &= -1ULL >> s; + else + n[0] &= -1ULL << s; } else { const char * lp = x.data + x.size - 8; memcpy(&n[0], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[0] >>= s; - else - n[0] <<= s; + n[0] >>= s; + else + n[0] <<= s; } auto res = hash(k8); auto buck = getBucketFromHash(res); @@ -138,9 +138,9 @@ class TwoLevelStringHashTable : private boost::noncopyable const char * lp = x.data + x.size - 8; memcpy(&n[1], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[1] >>= s; + n[1] >>= s; else - n[1] <<= s; + n[1] <<= s; auto res = hash(k16); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); @@ -152,9 +152,9 @@ class TwoLevelStringHashTable : private boost::noncopyable const char * lp = x.data + x.size - 8; memcpy(&n[2], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[2] >>= s; + n[2] >>= s; else - n[2] <<= s; + n[2] <<= s; auto res = hash(k24); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); From 11016d4c5f36fa39a36c2c2b6c0eec7c1c3dfd5f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 25 Jul 2023 16:46:50 +0200 Subject: [PATCH 219/226] Revert "Rewrite uniq to count" --- src/Analyzer/Passes/UniqToCountPass.cpp | 198 ------------------ src/Analyzer/Passes/UniqToCountPass.h | 30 --- src/Analyzer/QueryTreePassManager.cpp | 2 - src/Core/Settings.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 7 - .../RewriteUniqToCountVisitor.cpp | 163 -------------- src/Interpreters/RewriteUniqToCountVisitor.h | 30 --- .../test_rewrite_uniq_to_count/__init__.py | 0 .../test_rewrite_uniq_to_count/test.py | 127 ----------- 9 files changed, 558 deletions(-) delete mode 100644 src/Analyzer/Passes/UniqToCountPass.cpp delete mode 100644 src/Analyzer/Passes/UniqToCountPass.h delete mode 100644 src/Interpreters/RewriteUniqToCountVisitor.cpp delete mode 100644 src/Interpreters/RewriteUniqToCountVisitor.h delete mode 100644 tests/integration/test_rewrite_uniq_to_count/__init__.py delete mode 100644 tests/integration/test_rewrite_uniq_to_count/test.py diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp deleted file mode 100644 index 7533a99107b2..000000000000 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ /dev/null @@ -1,198 +0,0 @@ -#include "UniqToCountPass.h" - -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace -{ - -bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; -} - -/// Extract the corresponding projection columns for group by node list. -/// For example: -/// SELECT a as aa, any(b) FROM table group by a; -> aa(ColumnNode) -NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) -{ - if (!query_node->hasGroupBy()) - return {}; - - NamesAndTypes result; - for (const auto & group_by_ele : query_node->getGroupByNode()->getChildren()) - { - const auto & projection_columns = query_node->getProjectionColumns(); - const auto & projection_nodes = query_node->getProjection().getNodes(); - - assert(projection_columns.size() == projection_nodes.size()); - - for (size_t i = 0; i < projection_columns.size(); i++) - { - if (projection_nodes[i]->isEqual(*group_by_ele)) - result.push_back(projection_columns[i]); - } - } - return result; -} - -/// Whether query_columns equals subquery_columns. -/// query_columns: query columns from query -/// subquery_columns: projection columns from subquery -bool nodeListEquals(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) -{ - if (query_columns.size() != subquery_columns.size()) - return false; - - for (const auto & query_column : query_columns) - { - auto find = std::find_if( - subquery_columns.begin(), - subquery_columns.end(), - [&](const auto & subquery_column) -> bool - { - if (auto * column_node = query_column->as()) - { - return subquery_column == column_node->getColumn(); - } - return false; - }); - - if (find == subquery_columns.end()) - return false; - } - return true; -} - -/// Whether subquery_columns contains all columns in subquery_columns. -/// query_columns: query columns from query -/// subquery_columns: projection columns from subquery -bool nodeListContainsAll(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) -{ - if (query_columns.size() > subquery_columns.size()) - return false; - - for (const auto & query_column : query_columns) - { - auto find = std::find_if( - subquery_columns.begin(), - subquery_columns.end(), - [&](const auto & subquery_column) -> bool - { - if (auto * column_node = query_column->as()) - { - return subquery_column == column_node->getColumn(); - } - return false; - }); - - if (find == subquery_columns.end()) - return false; - } - return true; -} - -} - -class UniqToCountVisitor : public InDepthQueryTreeVisitor -{ -public: - using Base = InDepthQueryTreeVisitor; - using Base::Base; - - void visitImpl(QueryTreeNodePtr & node) - { - auto * query_node = node->as(); - if (!query_node) - return; - - /// Check that query has only single table expression which is subquery - auto * subquery_node = query_node->getJoinTree()->as(); - if (!subquery_node) - return; - - /// Check that query has only single node in projection - auto & projection_nodes = query_node->getProjection().getNodes(); - if (projection_nodes.size() != 1) - return; - - /// Check that projection_node is a function - auto & projection_node = projection_nodes[0]; - auto * function_node = projection_node->as(); - if (!function_node) - return; - - /// Check that query single projection node is `uniq` or its variants - if (!matchFnUniq(function_node->getFunctionName())) - return; - - auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' - auto match_subquery_with_distinct = [&]() -> bool - { - if (!subquery_node->isDistinct()) - return false; - - /// uniq expression list == subquery projection columns - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjectionColumns())) - return false; - - return true; - }; - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' - auto match_subquery_with_group_by = [&]() -> bool - { - if (!subquery_node->hasGroupBy()) - return false; - - /// uniq argument node list == subquery group by node list - auto group_by_columns = extractProjectionColumnsForGroupBy(subquery_node); - - if (!nodeListEquals(uniq_arguments_nodes, group_by_columns)) - return false; - - /// subquery projection columns must contain all columns in uniq argument node list - if (!nodeListContainsAll(uniq_arguments_nodes, subquery_node->getProjectionColumns())) - return false; - - return true; - }; - - /// Replace uniq of initial query to count - if (match_subquery_with_distinct() || match_subquery_with_group_by()) - { - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); - - function_node->resolveAsAggregateFunction(std::move(aggregate_function)); - function_node->getArguments().getNodes().clear(); - - /// Update projection columns - query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); - } - } -}; - - -void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) -{ - if (!context->getSettings().optimize_uniq_to_count) - return; - - UniqToCountVisitor visitor; - visitor.visit(query_tree_node); -} - -} diff --git a/src/Analyzer/Passes/UniqToCountPass.h b/src/Analyzer/Passes/UniqToCountPass.h deleted file mode 100644 index 4992d524e5e7..000000000000 --- a/src/Analyzer/Passes/UniqToCountPass.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** Optimize `uniq` and its variants(except uniqUpTo) into `count` over subquery. - * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to - * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' - * - * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to - * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' - * - * Note that we can rewrite all uniq variants except uniqUpTo. - */ -class UniqToCountPass final : public IQueryTreePass -{ -public: - String getName() override { return "UniqToCount"; } - - String getDescription() override - { - return "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause."; - } - - void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; -}; - -} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index dd75b0f586d8..a6da2a666150 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -247,7 +246,6 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ead00cafb4d..8bebef5fb00f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -776,7 +776,6 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8402165b62b0..fc3ea3a13ca2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include #include @@ -427,12 +426,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr); } - if (settings.optimize_uniq_to_count) - { - RewriteUniqToCountMatcher::Data data_rewrite_uniq_count; - RewriteUniqToCountVisitor(data_rewrite_uniq_count).visit(query_ptr); - } - JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view); bool got_storage_from_query = false; diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp deleted file mode 100644 index 7445068207ab..000000000000 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ /dev/null @@ -1,163 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -using Aliases = std::unordered_map; - -namespace -{ - -bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; -} - -bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, const Aliases & alias) -{ - if (lhs->getTreeHash() == rhs->getTreeHash()) - { - return true; - } - else - { - auto * lhs_idf = lhs->as(); - auto * rhs_idf = rhs->as(); - if (lhs_idf && rhs_idf) - { - /// compound identifiers, such as: - if (lhs_idf->shortName() == rhs_idf->shortName()) - return true; - - /// translate alias - if (alias.find(lhs_idf->shortName()) != alias.end()) - lhs_idf = alias.find(lhs_idf->shortName())->second->as(); - - if (alias.find(rhs_idf->shortName()) != alias.end()) - rhs_idf = alias.find(rhs_idf->shortName())->second->as(); - - if (lhs_idf->shortName() == rhs_idf->shortName()) - return true; - } - } - return false; -} - -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) -{ - if (!lhs || !rhs) - return false; - if (lhs->children.size() != rhs->children.size()) - return false; - for (size_t i = 0; i < lhs->children.size(); i++) - { - if (!expressionEquals(lhs->children[i], rhs->children[i], alias)) - return false; - } - return true; -} - -/// Test whether lhs contains all expressions in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) -{ - if (!lhs || !rhs) - return false; - if (lhs->children.size() < rhs->children.size()) - return false; - for (const auto & re : rhs->children) - { - auto predicate = [&re, &alias](ASTPtr & le) { return expressionEquals(le, re, alias); }; - if (std::find_if(lhs->children.begin(), lhs->children.end(), predicate) == lhs->children.end()) - return false; - } - return true; -} - -} - -void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) -{ - auto * selectq = ast->as(); - if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) - return; - auto expr_list = selectq->select(); - if (!expr_list || expr_list->children.size() != 1) - return; - auto * func = expr_list->children[0]->as(); - if (!func || !matchFnUniq(func->name)) - return; - if (selectq->tables()->as()->children[0]->as()->children.size() != 1) - return; - auto * table_expr = selectq->tables() - ->as() - ->children[0] - ->as() - ->children[0] - ->as(); - if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) - return; - auto * subquery = table_expr->subquery->as(); - if (!subquery) - return; - auto * sub_selectq = subquery->children[0] - ->as()->children[0] - ->as()->children[0] - ->as(); - if (!sub_selectq) - return; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return; - - /// collect subquery select expressions alias - Aliases alias; - for (const auto & expr : sub_expr_list->children) - { - if (!expr->tryGetAlias().empty()) - alias.insert({expr->tryGetAlias(), expr}); - } - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' - auto match_subquery_with_distinct = [&]() -> bool - { - if (!sub_selectq->distinct) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as(), alias)) - return false; - return true; - }; - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' - auto match_subquery_with_group_by = [&]() -> bool - { - auto group_by = sub_selectq->groupBy(); - if (!group_by) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), group_by->as(), alias)) - return false; - /// subquery select expression list must contain all columns in uniq expression list - if (!expressionListContainsAll(sub_expr_list->as(), func->children[0]->as(), alias)) - return false; - return true; - }; - - if (match_subquery_with_distinct() || match_subquery_with_group_by()) - expr_list->children[0] = makeASTFunction("count"); -} - -} diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h deleted file mode 100644 index 94528ccf2ee3..000000000000 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include "Interpreters/TreeRewriter.h" - -namespace DB -{ - -class ASTFunction; - -/** Optimize `uniq` into `count` over subquery. - * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to - * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' - * - * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to - * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' - * - * Note that we can rewrite all uniq variants except uniqUpTo. - */ -class RewriteUniqToCountMatcher -{ -public: - struct Data {}; - static void visit(ASTPtr & ast, Data &); - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } -}; - -using RewriteUniqToCountVisitor = InDepthNodeVisitor; -} diff --git a/tests/integration/test_rewrite_uniq_to_count/__init__.py b/tests/integration/test_rewrite_uniq_to_count/__init__.py deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py deleted file mode 100644 index e38e57f5cee9..000000000000 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ /dev/null @@ -1,127 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - prepare() - yield cluster - finally: - shutdown() - cluster.shutdown() - - -def prepare(): - node.query( - """ - CREATE TABLE IF NOT EXISTS test_rewrite_uniq_to_count - ( - `a` UInt8, - `b` UInt8, - `c` UInt8 - ) - ENGINE = MergeTree - ORDER BY `a` - """ - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')" - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')" - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')" - ) - - -def shutdown(): - node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") - - -def check(query, result): - # old analyzer - query = query + " settings optimize_uniq_to_count = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN SYNTAX " + query) - - # new analyzer - query = query + ", allow_experimental_analyzer = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN QUERY TREE " + query) - - -def check_by_old_analyzer(query, result): - # only old analyzer - query = query + " settings optimize_uniq_to_count = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN SYNTAX " + query) - - -def test_rewrite_distinct(started_cluster): - # simple test - check( - "SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", - 3, - ) - - # test subquery alias - check( - "SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test compound column name - check( - "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - -def test_rewrite_group_by(started_cluster): - # simple test - check( - "SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a)", - 3, - ) - - # test subquery alias - check( - "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) From 413ec520b3027d9f377aa1929a2855429994ffe3 Mon Sep 17 00:00:00 2001 From: Sanjam Panda Date: Tue, 25 Jul 2023 18:54:27 +0200 Subject: [PATCH 220/226] fix code style --- src/Common/HashTable/TwoLevelStringHashTable.h | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index 0527ec67e6ed..54c208c5b603 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -113,20 +113,20 @@ class TwoLevelStringHashTable : private boost::noncopyable if ((reinterpret_cast(p) & 2048) == 0) { memcpy(&n[0], p, 8); - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::endian::native == std::endian::little) n[0] &= -1ULL >> s; else n[0] &= -1ULL << s; - } + } else { const char * lp = x.data + x.size - 8; memcpy(&n[0], lp, 8); - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::endian::native == std::endian::little) n[0] >>= s; else n[0] <<= s; - } + } auto res = hash(k8); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); @@ -139,9 +139,9 @@ class TwoLevelStringHashTable : private boost::noncopyable memcpy(&n[1], lp, 8); if constexpr (std::endian::native == std::endian::little) n[1] >>= s; - else + else n[1] <<= s; - auto res = hash(k16); + auto res = hash(k16); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m2, k16, res); @@ -153,9 +153,9 @@ class TwoLevelStringHashTable : private boost::noncopyable memcpy(&n[2], lp, 8); if constexpr (std::endian::native == std::endian::little) n[2] >>= s; - else + else n[2] <<= s; - auto res = hash(k24); + auto res = hash(k24); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m3, k24, res); From 59db21941034a287eea6c1016ed2ca83e6772774 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 25 Jul 2023 19:21:41 +0200 Subject: [PATCH 221/226] Fix possible error "Cannot drain connections: cancel first" --- src/QueryPipeline/RemoteQueryExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index cd6f65b7b435..198c3265a840 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -591,8 +591,8 @@ void RemoteQueryExecutor::finish() /// Send the request to abort the execution of the request, if not already sent. tryCancel("Cancelling query because enough data has been read"); - /// If connections weren't created yet or query wasn't sent, nothing to do. - if (!connections || !sent_query) + /// If connections weren't created yet, query wasn't sent or was already finished, nothing to do. + if (!connections || !sent_query || finished) return; /// Get the remaining packets so that there is no out of sync in the connections to the replicas. From d78b3e560f13a6ba8b85b76e2f0d56bea44f2c62 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Jul 2023 17:45:13 +0000 Subject: [PATCH 222/226] Fix 02497_trace_events_stress_long again --- .../0_stateless/02497_trace_events_stress_long.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 91f6a9bb541d..c111ed40a296 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -45,4 +45,11 @@ thread2 $TIMEOUT >/dev/null & wait -$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' \ No newline at end of file +for _ in {1..10} +do + # process list is cleaned after everything is sent to client + # so this check can be run before process list is cleaned + # to avoid spurious failures we retry the check couple of times + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break + sleep 1 +done \ No newline at end of file From 20300804b13187447e8677573b46ee70175c98cc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 00:01:46 +0300 Subject: [PATCH 223/226] Update test.py --- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index d1373d44d0fa..67dd03098e9a 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -55,16 +55,7 @@ def insert(value): node1.query(f"INSERT INTO test_table VALUES ({value})") with PartitionManager() as pm: - assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) - # iptables rules must be applied immediately, but looks like sometimes they are not... - assert_eq_with_retry( - node1, - "select count() from remote('node1,node2', 'system.one')", - "1\n", - settings={"skip_unavailable_shards": 1}, - ) - p.map(insert, range(1, 50)) # Start shutdown async From d85f9ddb35f02564fe9d04f20f0a3451530a2b4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 00:03:08 +0300 Subject: [PATCH 224/226] Update parallel_skip.json --- tests/integration/parallel_skip.json | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 407fe7d1b012..1075fbaa0f8f 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -69,6 +69,8 @@ "test_server_reload/test.py::test_remove_tcp_port", "test_keeper_map/test.py::test_keeper_map_without_zk", + + "test_replicated_merge_tree_wait_on_shutdown/test.py::test_shutdown_and_wait", "test_http_failover/test.py::test_url_destination_host_with_multiple_addrs", "test_http_failover/test.py::test_url_invalid_hostname", From 93e10077bad715235dfe7d4da6d103ffbb30f55a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 26 Jul 2023 11:53:19 +0200 Subject: [PATCH 225/226] Fix attaching gdb in stress tests (#51445) * Fix attaching gdb in stress tests * Fix * Update run.sh * Try remove run_with_retry * Return run_with_retry * Don't set -e in run_with_retry if it was't set before * Update tests/ci/utils.lib * Fix bash --------- Co-authored-by: Alexander Tokmakov --- docker/test/stress/run.sh | 3 ++- docker/test/upgrade/run.sh | 1 + tests/ci/stress_tests.lib | 2 -- tests/ci/utils.lib | 11 +++++++++-- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 4926967d2d2a..9217fcfddd97 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -14,6 +14,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib +source /usr/share/clickhouse-test/ci/attach_gdb.lib source /usr/share/clickhouse-test/ci/stress_tests.lib install_packages package_folder @@ -52,7 +53,7 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & start -shellcheck disable=SC2086 # No quotes because I want to split it into words. +# shellcheck disable=SC2086 # No quotes because I want to split it into words. /s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index b80613093428..73a2965bf449 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -16,6 +16,7 @@ ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_pre # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib +source /usr/share/clickhouse-test/ci/attach_gdb.lib source /usr/share/clickhouse-test/ci/stress_tests.lib azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & diff --git a/tests/ci/stress_tests.lib b/tests/ci/stress_tests.lib index 190f3f39f9ea..85b376ac39d9 100644 --- a/tests/ci/stress_tests.lib +++ b/tests/ci/stress_tests.lib @@ -9,8 +9,6 @@ FAIL="\tFAIL\t\\N\t" FAILURE_CONTEXT_LINES=100 FAILURE_CONTEXT_MAX_LINE_WIDTH=300 -source attach_gdb.lib - function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. diff --git a/tests/ci/utils.lib b/tests/ci/utils.lib index b5ce4ae0d783..c90b7ebe6f68 100644 --- a/tests/ci/utils.lib +++ b/tests/ci/utils.lib @@ -2,6 +2,11 @@ function run_with_retry() { + if [[ $- =~ e ]]; then + set_e=true + else + set_e=false + fi set +e local total_retries="$1" @@ -12,7 +17,9 @@ function run_with_retry() until [ "$retry" -ge "$total_retries" ] do if "$@"; then - set -e + if $set_e; then + set -e + fi return else retry=$((retry + 1)) @@ -26,4 +33,4 @@ function run_with_retry() function fn_exists() { declare -F "$1" > /dev/null; -} \ No newline at end of file +} From 04180549b094c231a01642cb70fa051bed2f7abb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 13:15:58 +0200 Subject: [PATCH 226/226] Fix possible double-free in Aggregator (#52439) --- src/Interpreters/Aggregator.cpp | 6 ++++-- .../test.py | 2 +- .../0_stateless/02355_control_block_size_in_aggregator.sql | 3 ++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c7d4b87694be..36cd32910b51 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2020,7 +2020,8 @@ template NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = params.max_block_size; + /// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated + const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1; const bool final = true; ConvertToBlockRes res; @@ -2097,7 +2098,8 @@ template Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = params.max_block_size; + /// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated + const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1; const bool final = false; ConvertToBlockRes res; diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index faa38af65336..e66631460f7a 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -68,7 +68,7 @@ def test_distributed_directory_monitor_split_batch_on_failure_OFF(started_cluste settings={ # max_memory_usage is the limit for the batch on the remote node # (local query should not be affected since 30MB is enough for 100K rows) - "max_memory_usage": "30Mi", + "max_memory_usage": "20Mi", "max_untracked_memory": "0", }, ) diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql index b4754c6d6fe2..f9f9661a7c4d 100644 --- a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql @@ -1,6 +1,7 @@ SET max_block_size = 4213; -SELECT DISTINCT (blockSize() <= 4213) +--- We allocate space for one more row in case nullKeyData is present. +SELECT DISTINCT (blockSize() <= 4214) FROM ( SELECT number